
"How can I add a new table to start capturing its changes?"
This is one of the most common questions from our community.
If you ask The Ultimate Hitchhiker’s Guide to the Galaxy, the answer to everything is 42. Sadly, in the real world, it’s not quite that simple. The real answer is: it depends.
With this article, I’d like not only to provide an answer by walking through different scenarios, but also to explain the reasons behind it.
Introduction
Before exploring all the possibilities, we first need to define some basic Debezium concepts.
How does Debezium know which tables to capture?
Debezium offers a set of configuration properties to decide which databases, schemas, or tables should be captured. How these filters apply depends on the source database.
- MySQL / MariaDB
-
In these databases, the term database is basically the same as schema. So here you can filter by
database.include.list
andtable.include.list
, but there is no separate schema filter. - PostgreSQL, Oracle, SQL Server, Db2
-
A connector instance always captures from a single database. Inside that database, you can filter by
schema.include.list
andtable.include.list
.
The include properties are:
- database.include.list
-
An optional, comma-separated list of regular expressions that match database names. By default, all databases are included (where this makes sense).
- schema.include.list
-
An optional, comma-separated list of regular expressions that match schema names. By default, all non-system schemas are included.
- table.include.list
-
An optional, comma-separated list of regular expressions that match fully-qualified table identifiers. Each identifier is of the form schemaName.tableName. By default, all non-system tables are included.
There are also exclude counterparts if you want the opposite logic:
- database.exclude.list
-
An optional, comma-separated list of database names to exclude.
- schema.exclude.list
-
An optional, comma-separated list of schema names to exclude.
- table.exclude.list
-
An optional, comma-separated list of fully-qualified table identifiers to exclude.
At the end, it’s the combination of these properties that tells Debezium which tables it will capture events from. Remember: you can use either the include properties or the exclude properties, but never both at the same time.
How Debezium gets table schema information?
Debezium, in order to start streaming data changes from a table, needs to know its schema so it can provide information about the table’s structure. Normally, Debezium learns about table schemas while streaming so it can correctly interpret events. The exact mechanism depends on the database connector:
- DDL statements
-
Majority of connectors, such as MariaDB, MySQL, and Oracle, reads DDL statements from the transaction log, parses them, and update an in-memory representation of each table’s schema.
- Metadata messages
-
PostgreSQL uses Logical Decoding to send relation (table) metadata messages. Debezium receives a relation message (
R
) before the first change event for a table is sent, whenever a schema change occurs, or when replication resumes. These messages allow Debezium to know which table a change belongs to, the column order, data types, and other structural information. - JDBC queries
-
For connectors such as SQL Server and Db2, if a change event arrives for a table that is not yet in the in-memory schema representation, Debezium directly queries the database to retrieve the table schema.
For connectors that do not rely on metadata messages, the schema information is also persisted so that it can survive a connector restart or failure. For this purpose, there are several properties prefixed with schema.history.internal.
that manage configuration in this area. Of particular interest this the property:
schema.history.internal.store.only.captured.tables.ddl
-
Specifies whether the connector records schema structures from all tables in a schema or database, or only from tables that are designated for capture.
There is also a related property:
schema.history.internal.store.only.captured.databases.ddl
-
Specifies whether the connector records schema structures from all databases or only from databases that are designated for capture.
For now remember it, we will see in detail what role it has later.
Now, let’s take a step back: what happens when a connector is started for the first time?
Before emitting events for a table, Debezium needs to know the table structure. Even though this information could be obtained from a DDL change while streaming, it’s unlikely that the DDL is still present in the logs when the connector starts for the very first time. This is why Debezium requires capturing the database schema at first start-up.
As a side note, the initial snapshot exists primarily to capture the current data present in the database, since not all data may be available in the transaction log when Debezium starts for the very first time. Capturing the schema is required so that Debezium knows the table structures. Even if you don’t want to capture the existing data, schema capture cannot be disabled. You can, however, skip capturing the data itself by using the no_data
snapshot mode.
That said, you should start to understand why the answer is “it depends.”
Let’s walk through a couple of common scenarios you may run into.
Scenario 1
Let’s start with an easy scenario: you’ve configured the connector to capture all available tables in the database. This can be done either by setting the *.include.list
properties accordingly, or simply by leaving both include and exclude configurations unset — in that case, Debezium captures everything by default.
First of all, in this scenario, the schema.history.internal.store.only.captured.tables.ddl
property has no impact, since all table schemas are captured anyway.
If you create a new table while Debezium is streaming, it captures the DDL and updates its internal schema representation. When DML changes occur on that table, Debezium starts emitting change events immediately, since the table structure is already known.
You need to check that the database configuration for CDC is set up so that a new table is automatically enabled for CDC. This depends on the connector specifics. For example, in PostgreSQL, this means that publication.autocreate.mode is either all_tables (the default) or disabled, (and you manually created it for all tables). |
Scenario 2
Suppose now that we are in a scenario where the *.include.list
parameters are set so that Debezium captures only a subset of the tables available in the database. If you leave schema.history.internal.store.only.captured.tables.ddl
untouched (false
by default), Debezium will retrieve all table schemas during the initial snapshot.
So, if you want to start capturing changes from a table that wasn’t previously captured, you need to:
-
Edit the
table.include.list
property to specify the tables you want to capture. -
Restart the connector.
-
[Optional] Initiate an
incremental snapshot
if you want to capture existing data from the newly added tables.
The process is slightly more complex if you set schema.history.internal.store.only.captured.tables.ddl
to true
. In that case, since Debezium has no information about the table you want to add, you need to explicitly tell the connector to retrieve it.
The procedure is:
-
Stop the connector.
-
Remove the internal database schema history topic specified by the
schema.history.internal.kafka.topic
property. -
Set the
snapshot.mode
property torecovery
(orno_data
for connectors that don’t support recovery, like PostgreSQL). -
Add the table to the capture list.
-
Restart the connector.
Basically, this puts Debezium in a “first run” mode just for the schema. No existing data will be re-snapshotted.
The recovery option should be used with care as it assumes there have been no schema changes since the connector last stopped, otherwise some events occurred during the gap may be processed with an incorrect schema and corrupted. |
We have implemented an experimental feature for Oracle that automatically registers the schema for a table in-flight when a DML occurs. This means that if Debezium observes an insert, update, or delete event for a table whose schema is not yet captured and the table matches the include list, it will attempt to register the current schema and process the event using that schema.
Wait! What about MongoDB?
Due to its schemaless nature, MongoDB doesn’t require you to predefine a schema like relational databases do. So if you want to add a new collection to be captured, you just need to:
-
Ensure that the configured
capture.scope
is set to include the desired collection. -
Edit the
collection.include.list
to add the new collection.
Conclusion
We have seen how to add a new table to the captured list in different scenarios, and also explored the reasons behind the required steps. I hope this improves your Debezium knowledge and helps you manage it more confidently.
The Debezium team is continually seeking ways to enhance the project and its usability, and this article is part of that ongoing effort.
Since the easiest features are the ones you don’t need to explain, we’re considering adding an out-of-the-box signal action to trigger a schema refresh for specific tables. This would make the process of adding a new table to the capture list smoother and more straightforward.
Fiore Mario Vitale
Active in the open-source community with contributions to various projects, Mario is deeply involved in developing Debezium, the distributed platform for change data capture. During his career he developed extensive experience in event-driven architectures heavily influenced by data. Throughout his career, Mario has predominantly focused on data-intensive software and product development, which has heightened his sensitivity to developer experience and data-driven applications. Beyond his professional pursuits, Mario finds his sweet spot where tech and his personal interests come together. He loves taking photos, especially when he manages to freeze a beautiful moment. He's also passionate about motorsports and racing. When he's not coding, you can often find him exploring the great outdoors on his mountain bike, fueling his passion for adventure.

About Debezium
Debezium is an open source distributed platform that turns your existing databases into event streams, so applications can see and respond almost instantly to each committed row-level change in the databases. Debezium is built on top of Kafka and provides Kafka Connect compatible connectors that monitor specific database management systems. Debezium records the history of data changes in Kafka logs, so your application can be stopped and restarted at any time and can easily consume all of the events it missed while it was not running, ensuring that all events are processed correctly and completely. Debezium is open source under the Apache License, Version 2.0.
Get involved
We hope you find Debezium interesting and useful, and want to give it a try. Follow us on Twitter @debezium, chat with us on Zulip, or join our mailing list to talk with the community. All of the code is open source on GitHub, so build the code locally and help us improve ours existing connectors and add even more connectors. If you find problems or have ideas how we can improve Debezium, please let us know or log an issue.