Debezium Signal Table - Create schema when performing ad hoc snapshot

1,429 views
Skip to first unread message

Nathan Smit

unread,
Jan 7, 2022, 4:23:20 AM1/7/22
to debezium
Hi there,

I'm using Debezium Server 1.8 and the Debezium Oracle Connector.  I'm currently using signals in order to perform snapshots.  This works pretty well for existing tables, however,  when trying to snapshot a new table the ad hoc snapshot doesn't work as the schema doesn't exist.

Ideally, I'd like to be able to add a new table without actually restarting the connector.  Is that possible with signals currently?  I know with the Kafka setup it's easier to update the config while the connector is running but haven't been able to find a way to do this with Debezium server?

jiri.p...@gmail.com

unread,
Jan 10, 2022, 4:33:41 AM1/10/22
to debezium

Chris Cranford

unread,
Jan 10, 2022, 9:36:33 AM1/10/22
to debe...@googlegroups.com, jiri.p...@gmail.com
Hi Nathan -

You mentioned without restarting the connector so this has me curious, how's your "table.include.list" defined?  Is it defined at all and if so does it use regex or explicit table names?  If you use explicit table names, you're going to have to restart the connector because the new table will need to be part of the include list for changes to be streamed.  If you're using regex values in your include list, then technically no restart should be necessary as long as the new table is matched by the existing regex patterns.

Chris
--
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/df35b03f-6d6e-4729-82d2-a16fba0045d6n%40googlegroups.com.

Nathan Smit

unread,
Jan 13, 2022, 7:06:23 AM1/13/22
to debezium
Hey Chris,

I didn't explain this super well.  So I have explicitly named tables which will look something like this:

debezium.source.table.include.list=PICK.PICK,DEBEZIUM.DEBEZIUM_SIGNAL
debezium.source.signal.data.collection=GLODCP.DEBEZIUM.DEBEZIUM_SIGNAL

Now  I want to add a new table called e.g. PICK.PICKER.  My thinking was that I could implement as follows:
1)  Stop Debezium Server
2)  Update the include list to include PICK.PICKER
3)  Insert an incremental snapshot record into DEBEZIUM_SIGNAL
4)  Start up Debezium Server again

If I do the above though, the snapshot fails as the schema hasn't yet been created (I would typically want to do this after hours when there won't be activity on the table, but it could also be that the table is only updated once a week or whatever).  I have to wait for records to be inserted into the table so that a schema is created and can only then run the snapshot (otherwise some other hacky thing of first creating all the schemas again in file_database_history or something like that).  In other words, there's still some of the messiness to sort through that was there before signals

The part about having to restart the connector was a separate thing where it'd be nice to be able to add tables to debezium server without stopping and starting the server.  So in this blog for instance you can update the config using a curl request which I don't think is possible in Debezium Server https://debezium.io/blog/2021/10/07/incremental-snapshots/.   That's much more of a nice-to-have though.  It's more a matter of being able to do the steps in 1-4.

Chris Cranford

unread,
Jan 14, 2022, 10:46:51 AM1/14/22
to debe...@googlegroups.com, Nathan Smit
Hi Nathan -

Thanks for the more clear explanation. 

So yes we are aware there are some rough edges here with new tables using incremental snapshots.  The link that Jiri provided below is our conversation about this very thing on Zulip.  Our consensus is to add a flag in signal that indicates that the Incremental Snapshot process needs to perform a table schema sync prior to the incremental snapshot.  That should alleviate the problem; however that won't likely be available until 1.9.0.Alpha1 at the earliest.  In the meantime, there is an alternative you could try.

    1) Stop Debezium
    2) Update the include list to include your new table
    3) Change snapshot.mode to "schema_only_recovery"
    4) Start the connector, let it perform the recovery
    5) Stop and restart the connector by changing the snapshot.mode back to its original value
    6) Insert your incremental snapshot record
    7) Let Debezium perform the incremental snapshot

Regarding the on-the-fly configuration changes without a restart, the blog you referenced is working with Kafka Connect's REST API.  That's why the CURL example exists.  Unfortunately nothing like that exists in Debezium server but I could see an argument to consider adding something that is maybe a REST-lite component.  You're welcomed to open a Jira feature request and start a dialog.  Contributions are also very much welcomed :).

Chris

Nathan Smit

unread,
Jan 27, 2022, 9:26:48 AM1/27/22
to debezium
Hey there,

I'm trying to implement the workaround Chris gave but it doesn't seem to be working for me.  So I start off with the below two tables which are in the file_database_history as expected:
file_database_history_on_first_start.png

I then stop debezium server and make the below config changes:
config_change.png

Then I start up debezium server again, but...nothing happens.  As far as I can tell the schema_only_recovery isn't doing anything and if I try run a snapshot after restarting the connector again I still get the below error.  Not sure what I'm doing wrong here...

:"Schema not found for table 'GLODCP.LOOKUP.BRANCH_TEST_DUPLICATE', known tables [GLODCP.DEBEZIUM.DEBEZIUM_SIGNAL, GLODCP.LOOKUP.BRANCH_TEST_LIMITED]"





debz_output.txt

Nathan Smit

unread,
Jan 27, 2022, 9:28:37 AM1/27/22
to debezium
I attached my output logs as well to the previous message

Chris Cranford

unread,
Jan 27, 2022, 1:03:10 PM1/27/22
to debe...@googlegroups.com, Nathan Smit
Nathan, can you try one more thing; use the same steps but just before the restart can you either point to a new db history topic or delete the existing one?  I didn't realize that the "schema_only_recovery" option won't be triggered unless the history topic does not exist.  Let me know if that helps.

Nathan Smit

unread,
Jan 27, 2022, 1:57:42 PM1/27/22
to debezium
Hey Chris, so I've tried a few different variations.  Deleting the file_database_history, as well as filling it with gibberish, but neither solved the issue.  I don't use Kafka though for the file_database_history and I'm instead writing it to disk.  Could that be the issue?  I could try out writing the history to a kafka topic as an experiment.  At this point I'm trying to figure out the specific way to break the connector that will allow the recovery to work :D

Chris Cranford

unread,
Jan 27, 2022, 2:07:13 PM1/27/22
to debe...@googlegroups.com, Nathan Smit
Hi Nathan

If you're using the file backing store, deleting the file should trigger the behavior.  I looked in this thread and you said you are on 1.8; so I can find it strange (unless there is a bug) that doesn't trigger this.  Looking at the connector, one of the first steps the connector does even before the snapshot is:

    validateAndLoadDatabaseHistory(connectorConfig, partition, previousOffset, schema);

Inside that method, there is a check that's like this:

    if (!schema.historyExists()) {
        LOGGER.warn("Database history was not found but was expected");
        if (config.getSnapshotMode().shouldSnapshotOnSchemaError()) {
            /* this should be triggered and we should write a message */
            LOGGER.info("The db-history topic is missing but we are in SCHEMA_ONLY_RECOVERY snapshot mode.
        }
    }

At least in your logs I see:

    Started database history recovery
    Finished database history recovery of 2 change(s) in 30 ms

So could you try deleting the history topic file and attaching the log of that for me after a restart?

Thanks
Chris

Nathan Smit

unread,
Jan 27, 2022, 2:35:03 PM1/27/22
to debezium
No problem.  So just to confirm my steps here, I'm running Debezium Server with the two tables and doing an initial snapshot.  Then I'm stopping debezium server, renaming the old file_database_history file, adding my table and changing the snapshot mode to schema_only_recovery.  When I start Debezium Server again, it creates an empty file_database_history file like so:

debeziumserverdata.png

I've attached the logging from the start up after I deleted the file.  Interestingly, if I then try to run a snapshot on one of the original tables it fails so it is indeed picking up that I now have an empty file_database_history which makes me think there must be some combination of steps that will work.

{"timestamp":"2022-01-27T21:33:45.648+02:00","sequence":192,"loggerClassName":"org.slf4j.impl.Slf4jLogger","loggerName":"io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource","level":"WARN","message":"Schema not found for table 'GLODCP.LOOKUP.BRANCH_TEST_LIMITED', known tables [GLODCP.DEBEZIUM.DEBEZIUM_SIGNAL]","threadName":"debezium-oracleconnector-glodc_schema-change-event-source-coordinator","threadId":41,"mdc":{"dbz.connectorName":"glodc_schema","dbz.connectorType":"Oracle","dbz.connectorContext":"streaming"},"ndc":"","hostName":"glodbnstrp1.pepstores.com","processName":"io.debezium.server.Main","processId":11106}

logging.txt

Nathan Smit

unread,
Jan 27, 2022, 4:24:58 PM1/27/22
to debezium
Just in addition to the above, I can get a snapshot to run with the extremely long and complicated list of steps below.  Not sure why.  If I swap out some of the steps it doesn't work so yeah.  Very odd.  Will experiment more tomorrrow as well.

1)  Start Debezium Server and Perform Initial snapshot
2)  Stop Debezium Server
3)  Rename both offset.dat and file_database_history
4)  set snapshot mode to schema_only, add table to include_list and start Debezium Server
5)  stop Debezium server after schema snapshot completes
7)  replace current offset.dat with previously saved version
8)  set snapshot to schema_only_recovery
9)  Insert signal record to debezium.debezium_signal for the new table
10)  start debezium server (snapshot will fail as schema doesn't exist for the table)
11)  stop debezium server
12)  start debezium server
13)  Insert signal record to debezium.debezium_signal
14)  Snapshot starts

Nathan Smit

unread,
Jan 31, 2022, 3:15:41 AM1/31/22
to debezium
Hey there, after doing some initial testing here:  I decided to try, instead of using the file storage for the history, using a kafka history topic.  I found that if I did the below, everything worked as expected:

1)  Start Debezium Server, history topic created in Kafka as expected
2)  Stop Debezium Server and delete history topic (I had to delete the topic for the schema_recovery to work as expected)
3)  Add table to config and switch to schema_only_recovery
4)  Start Debezium Server again.  If I check the schema history topic, the new table has been added

This is exactly as I'd expect it to work.  This leads me to think that there is a bug in this process when writing the history to disk

Chris Cranford

unread,
Jan 31, 2022, 8:17:52 AM1/31/22
to debe...@googlegroups.com, Nathan Smit
HI Nathan -

Could you please raise a Jira and we'll take a look then.  I agree, I don't think that's expected behavior at all.

Chris

Nathan Smit

unread,
Jan 31, 2022, 9:20:17 AM1/31/22
to debezium

Thanks Chris.  I'll create an issue on Jira.  I think I picked up some additional things while experimenting with the code as well.  One more question:  I tried to jippo that validateAndLoadDatabaseHistory so that it also passed that condition when  config.getSnapshotMode().shouldSnapshotOnSchemaError() = true.  I noticed though that this also seemed to reset the snapshot "state" of the connector i.e. when I then set to "initial" it did the snapshots again.  I just wanted to confirm if that's the expected behaviour or if that's because I broke something?  

In other words, after a schema recovery and then setting the snapshot.mode back to initial, is the expected behaviour for the initial snapshot to be done on all of the tables again?

Chris Cranford

unread,
Jan 31, 2022, 10:07:49 AM1/31/22
to debe...@googlegroups.com, Nathan Smit
If in the same series of steps you deleted your offsets or the offsets didn't exist, then yes an "initial" value for snapshot.mode would trigger a new snapshot.

Chris

Nathan Smit

unread,
Jan 31, 2022, 3:53:05 PM1/31/22
to debezium
ok, I think that may be a separate issue then as I have this when using the kafka history topic as well.  I find that if I:

1)  Run initial snapshot
2) Stop Debezium server, delete history topic, change made to schema_only_recovery
3)  Start Debezium server

Then I find that in the offset.dat file, the snapshot_completed changes to false and it triggers a snapshot when I switch back to initial (in this case I only deleted file_database_history.  I never deleted offsets.dat).

offsets_dat.png



Reply all
Reply to author
Forward
0 new messages