Debezium connector for SQL Server

Overview

The Debezium SQL Server connector is based on the change data capture feature that is available in SQL Server 2016 Service Pack 1 (SP1) and later Standard edition or Enterprise edition. The SQL Server capture process monitors designated databases and tables, and stores the changes into specifically created change tables that have stored procedure facades.

To enable the Debezium SQL Server connector to capture change event records for database operations, you must first enable change data capture on the SQL Server database. CDC must be enabled on both the database and on each table that you want to capture. After you set up CDC on the source database, the connector can capture row-level INSERT, UPDATE, and DELETE operations that occur in the database. The connector writes event records for each source table to a Kafka topic especially dedicated to that table. One topic exists for each captured table. Client applications read the Kafka topics for the database tables that they follow, and can respond to the row-level events they consume from those topics.

The first time that the connector connects to a SQL Server database or cluster, it takes a consistent snapshot of the schemas for all tables for which it is configured to capture changes, and streams this state to Kafka. After the snapshot is complete, the connector continuously captures subsequent row-level changes that occur. By first establishing a consistent view of all of the data, the connector can continue reading without having lost any of the changes that were made while the snapshot was taking place.

The Debezium SQL Server connector is tolerant of failures. As the connector reads changes and produces events, it periodically records the position of events in the database log (LSN / Log Sequence Number). If the connector stops for any reason (including communication failures, network problems, or crashes), after a restart the connector resumes reading the SQL Server CDC tables from the last point that it read.

Offsets are committed periodically. They are not committed at the time that a change event occurs. As a result, following an outage, duplicate events might be generated.

Fault tolerance also applies to snapshots. That is, if the connector stops during a snapshot, the connector begins a new snapshot when it restarts.

How the SQL Server connector works

To optimally configure and run a Debezium SQL Server connector, it is helpful to understand how the connector performs snapshots, streams change events, determines Kafka topic names, and uses metadata.

Snapshots

SQL Server CDC is not designed to store a complete history of database changes. For the Debezium SQL Server connector to establish a baseline for the current state of the database, it uses a process called snapshotting. The initial snapshot captures the structure and data of the tables in the database.

Default workflow that the Debezium SQL Server connector uses to perform an initial snapshot

The following workflow lists the steps that Debezium takes to create a snapshot. These steps describe the process for a snapshot when the snapshot.mode configuration property is set to its default value, which is initial. You can customize the way that the connector creates snapshots by changing the value of the snapshot.mode property. If you configure a different snapshot mode, the connector completes the snapshot by using a modified version of this workflow.

  1. Establish a connection to the database.

  2. Determine the tables to be captured. By default, the connector captures all non-system tables. To have the connector capture a subset of tables or table elements, you can set a number of include and exclude properties to filter the data, for example, table.include.list or table.exclude.list.

  3. Obtain a lock on the SQL Server tables for which CDC is enabled to prevent structural changes from occurring during creation of the snapshot. The level of the lock is determined by the snapshot.isolation.mode configuration property.

  4. Read the maximum log sequence number (LSN) position in the server’s transaction log.

  5. Capture the structure of all non-system, or all tables that are designated for capture. The connector persists this information in its internal database schema history topic. The schema history provides information about the structure that is in effect when a change event occurs.

    By default, the connector captures the schema of every table in the database that is in capture mode, including tables that are not configured for capture. If tables are not configured for capture, the initial snapshot captures only their structure; it does not capture any table data. For more information about why snapshots persist schema information for tables that you did not include in the initial snapshot, see Understanding why initial snapshots capture the schema for all tables.

  6. Release the locks obtained in Step 3, if necessary. Other database clients can now write to any previously locked tables.

  7. At the LSN position read in Step 4, the connector scans the tables to be captured. During the scan, the connector completes the following tasks:

    1. Confirms that the table was created before the snapshot began. If the table was created after the snapshot began, the connector skips the table. After the snapshot is complete, and the connector transitions to streaming, it emits change events for any tables that were created after the snapshot began.

    2. Produces a read event for each row that is captured from a table. All read events contain the same LSN position, which is the LSN position that was obtained in step 4.

    3. Emits each read event to the Kafka topic for the table.

  8. Records the successful completion of the snapshot in the connector offsets.

The resulting initial snapshot captures the current state of each row in the tables that are enabled for CDC. From this baseline state, the connector captures subsequent changes as they occur.

After the snapshot process begins, if the process is interrupted due to connector failure, rebalancing, or other reasons, the process restarts after the connector restarts.

After the connector completes the initial snapshot, it continues streaming from the position that it read in Step 4 so that it does not miss any updates.

If the connector stops again for any reason, after it restarts, it resumes streaming changes from where it previously left off.

Table 1. Settings for snapshot.mode connector configuration property
SettingDescription

always

Perform snapshot on each connector start. After the snapshot completes, the connector begins to stream event records for subsequent database changes.

initial

The connector performs a database snapshot as described in the default workflow for creating an initial snapshot. After the snapshot completes, the connector begins to stream event records for subsequent database changes.

initial_only

The connector performs a database snapshot and stops before streaming any change event records, not allowing any subsequent change events to be captured.

schema_only

Deprecated, see no_data.

no_data

The connector captures the structure of all relevant tables, performing all the steps described in the default snapshot workflow, except that it does not create READ events to represent the data set at the point of the connector’s start-up (Step 7.b).

recovery

Set this option to restore a database schema history topic that is lost or corrupted. After a restart, the connector runs a snapshot that rebuilds the topic from the source tables. You can also set the property to periodically prune a database schema history topic that experiences unexpected growth.

+ WARNING: Do not use this mode to perform a snapshot if schema changes were committed to the database after the last connector shutdown.

when_needed

After the connector starts, it performs a snapshot only if it detects one of the following circumstances:

  • It cannot detect any topic offsets.

  • A previously recorded offset specifies a log position that is not available on the server.

configuration_based

Set the snapshot mode to configuration_based to control snapshot behavior through the set of connector properties that have the prefix ‘snapshot.mode.configuration.based’.

custom

The custom snapshot mode lets you inject your own implementation of the io.debezium.spi.snapshot.Snapshotter interface. Set the snapshot.mode.custom.name configuration property to the name provided by the name() method of your implementation. The name is specified on the classpath of your Kafka Connect cluster. If you use the Debezium EmbeddedEngine, the name is included in the connector JAR file. For more information, see custom snapshotter SPI.

For more information, see snapshot.mode in the table of connector configuration properties.

Understanding why initial snapshots capture the schema history for all tables

The initial snapshot that a connector runs captures two types of information:

Table data

Information about INSERT, UPDATE, and DELETE operations in tables that are named in the connector’s table.include.list property.

Schema data

DDL statements that describe the structural changes that are applied to tables. Schema data is persisted to both the internal schema history topic, and to the connector’s schema change topic, if one is configured.

After you run an initial snapshot, you might notice that the snapshot captures schema information for tables that are not designated for capture. By default, initial snapshots are designed to capture schema information for every table that is present in the database, not only from tables that are designated for capture. Connectors require that the table’s schema is present in the schema history topic before they can capture a table. By enabling the initial snapshot to capture schema data for tables that are not part of the original capture set, Debezium prepares the connector to readily capture event data from these tables should that later become necessary. If the initial snapshot does not capture a table’s schema, you must add the schema to the history topic before the connector can capture data from the table.

In some cases, you might want to limit schema capture in the initial snapshot. This can be useful when you want to reduce the time required to complete a snapshot. Or when Debezium connects to the database instance through a user account that has access to multiple logical databases, but you want the connector to capture changes only from tables in a specific logic database.

Additional information

Capturing data from tables not captured by the initial snapshot (no schema change)

In some cases, you might want the connector to capture data from a table whose schema was not captured by the initial snapshot. Depending on the connector configuration, the initial snapshot might capture the table schema only for specific tables in the database. If the table schema is not present in the history topic, the connector fails to capture the table, and reports a missing schema error.

You might still be able to capture data from the table, but you must perform additional steps to add the table schema.

Prerequisites

  • You want to capture data from a table with a schema that the connector did not capture during the initial snapshot.

  • No schema changes were applied to the table between the LSNs of the earliest and latest change table entry that the connector reads. For information about capturing data from a new table that has undergone structural changes, see [db2-capturing-data-from-new-tables-with-schema-changes].

Procedure

  1. Stop the connector.

  2. Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property.

  3. Clear the offsets in the configured Kafka Connect offset.storage.topic. For more information about how to remove offsets, see the Debezium community FAQ.

    Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.

  4. Apply the following changes to the connector configuration:

    1. (Optional) Set the value of schema.history.internal.store.only.captured.tables.ddl to false. This setting causes the snapshot to capture the schema for all tables, and guarantees that, in the future, the connector can reconstruct the schema history for all tables.

      Snapshots that capture the schema for all tables require more time to complete.

    2. Add the tables that you want the connector to capture to table.include.list.

    3. Set the snapshot.mode to one of the following values:

      initial

      When you restart the connector, it takes a full snapshot of the database that captures the table data and table structures.
      If you select this option, consider setting the value of the schema.history.internal.store.only.captured.tables.ddl property to false to enable the connector to capture the schema of all tables.

      schema_only

      When you restart the connector, it takes a snapshot that captures only the table schema. Unlike a full data snapshot, this option does not capture any table data. Use this option if you want to restart the connector more quickly than with a full snapshot.

  5. Restart the connector. The connector completes the type of snapshot specified by the snapshot.mode.

  6. (Optional) If the connector performed a schema_only snapshot, after the snapshot completes, initiate an incremental snapshot to capture data from the tables that you added. The connector runs the snapshot while it continues to stream real-time changes from the tables. Running an incremental snapshot captures the following data changes:

    • For tables that the connector previously captured, the incremental snapsot captures changes that occur while the connector was down, that is, in the interval between the time that the connector was stopped, and the current restart.

    • For newly added tables, the incremental snapshot captures all existing table rows.

Capturing data from tables not captured by the initial snapshot (schema change)

If a schema change is applied to a table, records that are committed before the schema change have different structures than those that were committed after the change. When Debezium captures data from a table, it reads the schema history to ensure that it applies the correct schema to each event. If the schema is not present in the schema history topic, the connector is unable to capture the table, and an error results.

If you want to capture data from a table that was not captured by the initial snapshot, and the schema of the table was modified, you must add the schema to the history topic, if it is not already available. You can add the schema by running a new schema snapshot, or by running an initial snapshot for the table.

Prerequisites

  • You want to capture data from a table with a schema that the connector did not capture during the initial snapshot.

  • A schema change was applied to the table so that the records to be captured do not have a uniform structure.

Procedure

Initial snapshot captured the schema for all tables (store.only.captured.tables.ddl was set to false)

  1. Edit the table.include.list property to specify the tables that you want to capture.

  2. Restart the connector.

  3. Initiate an incremental snapshot if you want to capture existing data from the newly added tables.

Initial snapshot did not capture the schema for all tables (store.only.captured.tables.ddl was set to true)

If the initial snapshot did not save the schema of the table that you want to capture, complete one of the following procedures:

  • Procedure 1: Schema snapshot, followed by incremental snapshot

    In this procedure, the connector first performs a schema snapshot. You can then initiate an incremental snapshot to enable the connector to synchronize data.

    1. Stop the connector.

    2. Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property.

    3. Clear the offsets in the configured Kafka Connect offset.storage.topic. For more information about how to remove offsets, see the Debezium community FAQ.

      Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.

    4. Set values for properties in the connector configuration as described in the following steps:

      1. Set the value of the snapshot.mode property to schema_only.

      2. Edit the table.include.list to add the tables that you want to capture.

    5. Restart the connector.

    6. Wait for Debezium to capture the schema of the new and existing tables. Data changes that occurred any tables after the connector stopped are not captured.

    7. To ensure that no data is lost, initiate an incremental snapshot.

    Procedure 2: Initial snapshot, followed by optional incremental snapshot

    In this procedure the connector performs a full initial snapshot of the database. As with any initial snapshot, in a database with many large tables, running an initial snapshot can be a time-consuming operation. After the snapshot completes, you can optionally trigger an incremental snapshot to capture any changes that occur while the connector is off-line.

    1. Stop the connector.

    2. Remove the internal database schema history topic that is specified by the schema.history.internal.kafka.topic property.

    3. Clear the offsets in the configured Kafka Connect offset.storage.topic. For more information about how to remove offsets, see the Debezium community FAQ.

      Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.

    4. Edit the table.include.list to add the tables that you want to capture.

    5. Set values for properties in the connector configuration as described in the following steps:

      1. Set the value of the snapshot.mode property to initial.

      2. (Optional) Set schema.history.internal.store.only.captured.tables.ddl to false.

    6. Restart the connector. The connector takes a full database snapshot. After the snapshot completes, the connector transitions to streaming.

    7. (Optional) To capture any data that changed while the connector was off-line, initiate an incremental snapshot.

Ad hoc snapshots

By default, a connector runs an initial snapshot operation only after it starts for the first time. Following this initial snapshot, under normal circumstances, the connector does not repeat the snapshot process. Any future change event data that the connector captures comes in through the streaming process only.

However, in some situations the data that the connector obtained during the initial snapshot might become stale, lost, or incomplete. To provide a mechanism for recapturing table data, Debezium includes an option to perform ad hoc snapshots. You might want to perform an ad hoc snapshot after any of the following changes occur in your Debezium environment:

  • The connector configuration is modified to capture a different set of tables.

  • Kafka topics are deleted and must be rebuilt.

  • Data corruption occurs due to a configuration error or some other problem.

You can re-run a snapshot for a table for which you previously captured a snapshot by initiating a so-called ad-hoc snapshot. Ad hoc snapshots require the use of signaling tables. You initiate an ad hoc snapshot by sending a signal request to the Debezium signaling table.

When you initiate an ad hoc snapshot of an existing table, the connector appends content to the topic that already exists for the table. If a previously existing topic was removed, Debezium can create a topic automatically if automatic topic creation is enabled.

Ad hoc snapshot signals specify the tables to include in the snapshot. The snapshot can capture the entire contents of the database, or capture only a subset of the tables in the database. Also, the snapshot can capture a subset of the contents of the table(s) in the database.

You specify the tables to capture by sending an execute-snapshot message to the signaling table. Set the type of the execute-snapshot signal to incremental or blocking, and provide the names of the tables to include in the snapshot, as described in the following table:

Table 2. Example of an ad hoc execute-snapshot signal record
FieldDefaultValue

type

incremental

Specifies the type of snapshot that you want to run.
Currently, you can request incremental or blocking snapshots.

data-collections

N/A

An array that contains regular expressions matching the fully-qualified names of the table to be snapshotted.
The format of the names is the same as for the signal.data.collection configuration option.

additional-condition

N/A

An optional string, which specifies a condition based on the column(s) of the table(s), to capture a subset of the contents of the table(s).

This property is deprecated. To specify criteria for defining the subset of data that you want the snapshot to capture, use the additional-conditions parameter.

additional-conditions

N/A

An optional array that specifies a set of additional conditions that the connector evaluates to determine the subset of records to include in a snapshot.
Each additional condition is an object that specifies the criteria for filtering the data that an ad hoc snapshot captures. You can set the following parameters for each additional condition:

    data-collection

    The fully-qualified name of the table that the filter applies to. You can apply different filters to each table.

    filter

    Specifies column values that must be present in a database record for the snapshot to include it, for example, “color=’blue’”.

    The values that you assign to the filter parameter are the same types of values that you might specify in the WHERE clause of SELECT statements when you set the snapshot.select.statement.overrides property for a blocking snapshot. In earlier Debezium releases, an explicit filter parameter was not defined for snapshot signals; instead, filter criteria were implied by the values that were specified for the now deprecated additional-condition parameter.

surrogate-key

N/A

An optional string that specifies the column name that the connector uses as the primary key of a table during the snapshot process.

Triggering an ad hoc incremental snapshot

You initiate an ad hoc incremental snapshot by adding an entry with the execute-snapshot signal type to the signaling table. After the connector processes the message, it begins the snapshot operation. The snapshot process reads the first and last primary key values and uses those values as the start and end point for each table. Based on the number of entries in the table, and the configured chunk size, Debezium divides the table into chunks, and proceeds to snapshot each chunk, in succession, one at a time.

For more information, see Incremental snapshots.

Triggering an ad hoc blocking snapshot

You initiate an ad hoc blocking snapshot by adding an entry with the execute-snapshot signal type to the signaling table. After the connector processes the message, it begins the snapshot operation. The connector temporarily stops streaming, and then initiates a snapshot of the specified table, following the same process that it uses during an initial snapshot. After the snapshot completes, the connector resumes streaming.

For more information, see Blocking snapshots.

Incremental snapshots

SQL Server collations

Each SQL Server server or database is configured to use a specific collation, which determines how character data is stored, sorted, compared, and displayed. The sorting rules for some collation sets, such as the SQL Server collations (SQL_*) are not compatible with the Unicode sorting algorithm. In some cases, the incompatible sorting rules can lead to lost data when the connector runs an ad hoc snapshot. For example, if SQL Server is configured to send strings as Unicode (that is, the connection property sendStringParametersAsUnicode is set to true), the connector can skip records during the snapshot. To protect against lost data during an ad hoc snapshot, set the value of the driver.sendStringParametersAsUnicode connection string property to false.

For more information about using the sendStringParametersAsUnicode property, see the SQL Server connection properties documentation.

To provide flexibility in managing snapshots, Debezium includes a supplementary snapshot mechanism, known as incremental snapshotting. Incremental snapshots rely on the Debezium mechanism for sending signals to a Debezium connector. Incremental snapshots are based on the DDD-3 design document.

In an incremental snapshot, instead of capturing the full state of a database all at once, as in an initial snapshot, Debezium captures each table in phases, in a series of configurable chunks. You can specify the tables that you want the snapshot to capture and the size of each chunk. The chunk size determines the number of rows that the snapshot collects during each fetch operation on the database. The default chunk size for incremental snapshots is 1024 rows.

As an incremental snapshot proceeds, Debezium uses watermarks to track its progress, maintaining a record of each table row that it captures. This phased approach to capturing data provides the following advantages over the standard initial snapshot process:

  • You can run incremental snapshots in parallel with streamed data capture, instead of postponing streaming until the snapshot completes. The connector continues to capture near real-time events from the change log throughout the snapshot process, and neither operation blocks the other.

  • If the progress of an incremental snapshot is interrupted, you can resume it without losing any data. After the process resumes, the snapshot begins at the point where it stopped, rather than recapturing the table from the beginning.

  • You can run an incremental snapshot on demand at any time, and repeat the process as needed to adapt to database updates. For example, you might re-run a snapshot after you modify the connector configuration to add a table to its table.include.list property.

Incremental snapshot process

When you run an incremental snapshot, Debezium sorts each table by primary key and then splits the table into chunks based on the configured chunk size. Working chunk by chunk, it then captures each table row in a chunk. For each row that it captures, the snapshot emits a READ event. That event represents the value of the row when the snapshot for the chunk began.

As a snapshot proceeds, it’s likely that other processes continue to access the database, potentially modifying table records. To reflect such changes, INSERT, UPDATE, or DELETE operations are committed to the transaction log as per usual. Similarly, the ongoing Debezium streaming process continues to detect these change events and emits corresponding change event records to Kafka.

How Debezium resolves collisions among records with the same primary key

In some cases, the UPDATE or DELETE events that the streaming process emits are received out of sequence. That is, the streaming process might emit an event that modifies a table row before the snapshot captures the chunk that contains the READ event for that row. When the snapshot eventually emits the corresponding READ event for the row, its value is already superseded. To ensure that incremental snapshot events that arrive out of sequence are processed in the correct logical order, Debezium employs a buffering scheme for resolving collisions. Only after collisions between the snapshot events and the streamed events are resolved does Debezium emit an event record to Kafka.

Snapshot window

To assist in resolving collisions between late-arriving READ events and streamed events that modify the same table row, Debezium employs a so-called snapshot window. The snapshot windows demarcates the interval during which an incremental snapshot captures data for a specified table chunk. Before the snapshot window for a chunk opens, Debezium follows its usual behavior and emits events from the transaction log directly downstream to the target Kafka topic. But from the moment that the snapshot for a particular chunk opens, until it closes, Debezium performs a de-duplication step to resolve collisions between events that have the same primary key..

For each data collection, the Debezium emits two types of events, and stores the records for them both in a single destination Kafka topic. The snapshot records that it captures directly from a table are emitted as READ operations. Meanwhile, as users continue to update records in the data collection, and the transaction log is updated to reflect each commit, Debezium emits UPDATE or DELETE operations for each change.

As the snapshot window opens, and Debezium begins processing a snapshot chunk, it delivers snapshot records to a memory buffer. During the snapshot windows, the primary keys of the READ events in the buffer are compared to the primary keys of the incoming streamed events. If no match is found, the streamed event record is sent directly to Kafka. If Debezium detects a match, it discards the buffered READ event, and writes the streamed record to the destination topic, because the streamed event logically supersede the static snapshot event. After the snapshot window for the chunk closes, the buffer contains only READ events for which no related transaction log events exist. Debezium emits these remaining READ events to the table’s Kafka topic.

The connector repeats the process for each snapshot chunk.

The Debezium connector for SQL Server does not support schema changes while an incremental snapshot is running.

Triggering an incremental snapshot

Currently, the only way to initiate an incremental snapshot is to send an ad hoc snapshot signal to the signaling table on the source database.

You submit a signal to the signaling table as SQL INSERT queries.

After Debezium detects the change in the signaling table, it reads the signal, and runs the requested snapshot operation.

The query that you submit specifies the tables to include in the snapshot, and, optionally, specifies the type of snapshot operation. Currently supports the incremental and blocking types.

To specify the tables to include in the snapshot, provide a data-collections array that lists the tables or an array of regular expressions used to match tables, for example,

{"data-collections": ["public.MyFirstTable", "public.MySecondTable"]}

The data-collections array for an incremental snapshot signal has no default value. If the data-collections array is empty, Debezium detects that no action is required and does not perform a snapshot.

If the name of a table that you want to include in a snapshot contains a dot (.) in the name of the database, schema, or table, to add the table to the data-collections array, you must escape each part of the name in double quotes.

For example, to include a table that exists in the public schema and that has the name My.Table, use the following format: “public”.”My.Table”.

Prerequisites

Using a source signaling channel to trigger an incremental snapshot

  1. Send a SQL query to add the ad hoc incremental snapshot request to the signaling table:

    1. INSERT INTO <signalTable> (id, type, data) VALUES ('<id>', '<snapshotType>', '{"data-collections": ["<tableName>","<tableName>"],"type":"<snapshotType>","additional-conditions":[{"data-collection": "<tableName>", "filter": "<additional-condition>"}]}');

    For example,

    1. INSERT INTO myschema.debezium_signal (id, type, data) (1)
    2. values ('ad-hoc-1', (2)
    3. 'execute-snapshot', (3)
    4. '{"data-collections": ["schema1.table1", "schema2.table2"], (4)
    5. "type":"incremental", (5)
    6. "additional-conditions":[{"data-collection": "schema1.table1" ,"filter":"color=\'blue\'"}]}'); (6)

    The values of the id,type, and data parameters in the command correspond to the fields of the signaling table.

    The following table describes the parameters in the example:

    Table 3. Descriptions of fields in a SQL command for sending an incremental snapshot signal to the signaling table
    ItemValueDescription

    1

    myschema.debezium_signal

    Specifies the fully-qualified name of the signaling table on the source database.

    2

    ad-hoc-1

    The id parameter specifies an arbitrary string that is assigned as the id identifier for the signal request.
    Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string. Rather, during the snapshot, Debezium generates its own id string as a watermarking signal.

    3

    execute-snapshot

    The type parameter specifies the operation that the signal is intended to trigger.

    4

    data-collections

    A required component of the data field of a signal that specifies an array of table names or regular expressions to match table names to include in the snapshot.
    The array lists regular expressions which match tables by their fully-qualified names, using the same format as you use to specify the name of the connector’s signaling table in the signal.data.collection configuration property.

    5

    incremental

    An optional type component of the data field of a signal that specifies the type of snapshot operation to run.
    Currently supports the incremental and blocking types.
    If you do not specify a value, the connector runs an incremental snapshot.

    6

    additional-conditions

    An optional array that specifies a set of additional conditions that the connector evaluates to determine the subset of records to include in a snapshot.
    Each additional condition is an object with data-collection and filter properties. You can specify different filters for each data collection.
    * The data-collection property is the fully-qualified name of the data collection for which the filter will be applied. For more information about the additional-conditions parameter, see Ad hoc incremental snapshots with additional-conditions.

Ad hoc incremental snapshots with additional-conditions

If you want a snapshot to include only a subset of the content in a table, you can modify the signal request by appending an additional-conditions parameter to the snapshot signal.

The SQL query for a typical snapshot takes the following form:

  1. SELECT * FROM <tableName> ....

By adding an additional-conditions parameter, you append a WHERE condition to the SQL query, as in the following example:

  1. SELECT * FROM <data-collection> WHERE <filter> ....

The following example shows a SQL query to send an ad hoc incremental snapshot request with an additional condition to the signaling table:

  1. INSERT INTO <signalTable> (id, type, data) VALUES ('<id>', '<snapshotType>', '{"data-collections": ["<tableName>","<tableName>"],"type":"<snapshotType>","additional-conditions":[{"data-collection": "<tableName>", "filter": "<additional-condition>"}]}');

For example, suppose you have a products table that contains the following columns:

  • id (primary key)

  • color

  • quantity

If you want an incremental snapshot of the products table to include only the data items where color=blue, you can use the following SQL statement to trigger the snapshot:

  1. INSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.products"],"type":"incremental", "additional-conditions":[{"data-collection": "schema1.products", "filter": "color=blue"}]}');

The additional-conditions parameter also enables you to pass conditions that are based on more than one column. For example, using the products table from the previous example, you can submit a query that triggers an incremental snapshot that includes the data of only those items for which color=blue and quantity>10:

  1. INSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.products"],"type":"incremental", "additional-conditions":[{"data-collection": "schema1.products", "filter": "color=blue AND quantity>10"}]}');

The following example, shows the JSON for an incremental snapshot event that is captured by a connector.

Example: Incremental snapshot event message

  1. {
  2. "before":null,
  3. "after": {
  4. "pk":"1",
  5. "value":"New data"
  6. },
  7. "source": {
  8. ...
  9. "snapshot":"incremental" (1)
  10. },
  11. "op":"r", (2)
  12. "ts_ms":"1620393591654",
  13. "ts_us":"1620393591654547",
  14. "ts_ns":"1620393591654547920",
  15. "transaction":null
  16. }
ItemField nameDescription

1

snapshot

Specifies the type of snapshot operation to run.
Currently, the only valid options are blocking and incremental.
Specifying a type value in the SQL query that you submit to the signaling table is optional.
If you do not specify a value, the connector runs an incremental snapshot.

2

op

Specifies the event type.
The value for snapshot events is r, signifying a READ operation.

Using the Kafka signaling channel to trigger an incremental snapshot

You can send a message to the configured Kafka topic to request the connector to run an ad hoc incremental snapshot.

The key of the Kafka message must match the value of the topic.prefix connector configuration option.

The value of the message is a JSON object with type and data fields.

The signal type is execute-snapshot, and the data field must have the following fields:

Table 4. Execute snapshot data fields
FieldDefaultValue

type

incremental

The type of the snapshot to be executed. Currently Debezium supports the incremental and blocking types.
See the next section for more details.

data-collections

N/A

An array of comma-separated regular expressions that match the fully-qualified names of tables to include in the snapshot.
Specify the names by using the same format as is required for the signal.data.collection configuration option.

additional-condition

N/A

An optional string that specifies a condition that the connector evaluates to designate a subset of records to include in a snapshot.

This property is deprecated and should be replaced by the additional-conditions property.

additional-conditions

N/A

An optional array of additional conditions that specifies criteria that the connector evaluates to designate a subset of records to include in a snapshot.
Each additional condition is an object that specifies the criteria for filtering the data that an ad hoc snapshot captures. You can set the following parameters for each additional condition: data-collection:: The fully-qualified name of the table that the filter applies to. You can apply different filters to each table. filter:: Specifies column values that must be present in a database record for the snapshot to include it, for example, “color=’blue’”.

The values that you assign to the filter parameter are the same types of values that you might specify in the WHERE clause of SELECT statements when you set the snapshot.select.statement.overrides property for a blocking snapshot. In earlier Debezium releases, an explicit filter parameter was not defined for snapshot signals; instead, filter criteria were implied by the values that were specified for the now deprecated additional-condition parameter.

An example of the execute-snapshot Kafka message:

  1. Key = `test_connector`
  2. Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.table1", "schema1.table2"], "type": "INCREMENTAL"}}`

Ad hoc incremental snapshots with additional-conditions

Debezium uses the additional-conditions field to select a subset of a table’s content.

Typically, when Debezium runs a snapshot, it runs a SQL query such as:

SELECT * FROM _<tableName>_ …​.

When the snapshot request includes an additional-conditions property, the data-collection and filter parameters of the property are appended to the SQL query, for example:

SELECT * FROM _<data-collection>_ WHERE _<filter>_ …​.

For example, given a products table with the columns id (primary key), color, and brand, if you want a snapshot to include only content for which color='blue', when you request the snapshot, you could add the additional-conditions property to filter the content:

  1. Key = `test_connector`
  2. Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.products"], "type": "INCREMENTAL", "additional-conditions": [{"data-collection": "schema1.products" ,"filter":"color='blue'"}]}}`

You can use the additional-conditions property to pass conditions based on multiple columns. For example, using the same products table as in the previous example, if you want a snapshot to include only the content from the products table for which color='blue', and brand='MyBrand', you could send the following request:

  1. Key = `test_connector`
  2. Value = `{"type":"execute-snapshot","data": {"data-collections": ["schema1.products"], "type": "INCREMENTAL", "additional-conditions": [{"data-collection": "schema1.products" ,"filter":"color='blue' AND brand='MyBrand'"}]}}`

Stopping an incremental snapshot

You can also stop an incremental snapshot by sending a signal to the table on the source database. You submit a stop snapshot signal to the table by sending a SQL INSERT query.

After Debezium detects the change in the signaling table, it reads the signal, and stops the incremental snapshot operation if it’s in progress.

The query that you submit specifies the snapshot operation of incremental, and, optionally, the tables of the current running snapshot to be removed.

Prerequisites

Using a source signaling channel to stop an incremental snapshot

  1. Send a SQL query to stop the ad hoc incremental snapshot to the signaling table:

    1. INSERT INTO <signalTable> (id, type, data) values ('<id>', 'stop-snapshot', '{"data-collections": ["<tableName>","<tableName>"],"type":"incremental"}');

    For example,

    1. INSERT INTO myschema.debezium_signal (id, type, data) (1)
    2. values ('ad-hoc-1', (2)
    3. 'stop-snapshot', (3)
    4. '{"data-collections": ["schema1.table1", "schema2.table2"], (4)
    5. "type":"incremental"}'); (5)

    The values of the id, type, and data parameters in the signal command correspond to the fields of the signaling table.

    The following table describes the parameters in the example:

    Table 5. Descriptions of fields in a SQL command for sending a stop incremental snapshot signal to the signaling table
    ItemValueDescription

    1

    myschema.debezium_signal

    Specifies the fully-qualified name of the signaling table on the source database.

    2

    ad-hoc-1

    The id parameter specifies an arbitrary string that is assigned as the id identifier for the signal request.
    Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string.

    3

    stop-snapshot

    Specifies type parameter specifies the operation that the signal is intended to trigger.

    4

    data-collections

    An optional component of the data field of a signal that specifies an array of table names or regular expressions to match table names to remove from the snapshot.
    The array lists regular expressions which match tables by their fully-qualified names, using the same format as you use to specify the name of the connector’s signaling table in the signal.data.collection configuration property. If this component of the data field is omitted, the signal stops the entire incremental snapshot that is in progress.

    5

    incremental

    A required component of the data field of a signal that specifies the type of snapshot operation that is to be stopped.
    Currently, the only valid option is incremental.
    If you do not specify a type value, the signal fails to stop the incremental snapshot.

Using the Kafka signaling channel to stop an incremental snapshot

You can send a signal message to the configured Kafka signaling topic to stop an ad hoc incremental snapshot.

The key of the Kafka message must match the value of the topic.prefix connector configuration option.

The value of the message is a JSON object with type and data fields.

The signal type is stop-snapshot, and the data field must have the following fields:

Table 6. Execute snapshot data fields
FieldDefaultValue

type

incremental

The type of the snapshot to be executed. Currently Debezium supports only the incremental type.
See the next section for more details.

data-collections

N/A

An optional array of comma-separated regular expressions that match the fully-qualified names of the tables to include in the snapshot.
Specify the names by using the same format as is required for the signal.data.collection configuration option.

The following example shows a typical stop-snapshot Kafka message:

  1. Key = `test_connector`
  2. Value = `{"type":"stop-snapshot","data": {"data-collections": ["schema1.table1", "schema1.table2"], "type": "INCREMENTAL"}}`

Custom snapshotter SPI

For more advanced uses, you can fine-tune control of the snapshot by implementing one of the following interfaces:

io.debezium.snapshot.spi.Snapshotter

Controls whether the connector takes a snapshot.

io.debezium.snapshot.spi.SnapshotQuery

Controls how data is queried during a snapshot.

io.debezium.snapshot.spi.SnapshotLock

Controls whether the connector locks tables when taking a snapshot.

io.debezium.snapshot.spi.Snapshotter interface. All built-in snapshot modes implement this interface.

  1. /**
  2. * {@link Snapshotter} is used to determine the following details about the snapshot process:
  3. * <p>
  4. * - Whether a snapshot occurs. <br>
  5. * - Whether streaming continues during the snapshot. <br>
  6. * - Whether the snapshot includes schema (if supported). <br>
  7. * - Whether to snapshot data or schema following an error.
  8. * <p>
  9. * Although Debezium provides many default snapshot modes,
  10. * to provide more advanced functionality, such as partial snapshots,
  11. * you can customize implementation of the interface.
  12. * For more information, see the documentation.
  13. *
  14. *
  15. *
  16. */
  17. @Incubating
  18. public interface Snapshotter extends Configurable {
  19. /**
  20. * @return the name of the snapshotter.
  21. *
  22. *
  23. */
  24. String name();
  25. /**
  26. * @param offsetExists is {@code true} when the connector has an offset context (i.e. restarted)
  27. * @param snapshotInProgress is {@code true} when the connector is started, but a snapshot is already in progress
  28. *
  29. * @return {@code true} if the snapshotter should take a data snapshot
  30. */
  31. boolean shouldSnapshotData(boolean offsetExists, boolean snapshotInProgress);
  32. /**
  33. * @param offsetExists is {@code true} when the connector has an offset context (i.e. restarted)
  34. * @param snapshotInProgress is {@code true} when the connector is started, but a snapshot is already in progress
  35. *
  36. * @return {@code true} if the snapshotter should take a schema snapshot
  37. */
  38. boolean shouldSnapshotSchema(boolean offsetExists, boolean snapshotInProgress);
  39. /**
  40. * @return {@code true} if the snapshotter should stream after taking a snapshot
  41. */
  42. boolean shouldStream();
  43. /**
  44. * @return {@code true} whether the schema can be recovered if database schema history is corrupted.
  45. */
  46. boolean shouldSnapshotOnSchemaError();
  47. /**
  48. * @return {@code true} whether the snapshot should be re-executed when there is a gap in data stream.
  49. */
  50. boolean shouldSnapshotOnDataError();
  51. /**
  52. *
  53. * @return {@code true} if streaming should resume from the start of the snapshot
  54. * transaction, or {@code false} for when a connector resumes and takes a snapshot,
  55. * streaming should resume from where streaming previously left off.
  56. */
  57. default boolean shouldStreamEventsStartingFromSnapshot() {
  58. return true;
  59. }
  60. /**
  61. * Lifecycle hook called after the snapshot phase is successful.
  62. */
  63. default void snapshotCompleted() {
  64. // no operation
  65. }
  66. /**
  67. * Lifecycle hook called after the snapshot phase is aborted.
  68. */
  69. default void snapshotAborted() {
  70. // no operation
  71. }
  72. }

io.debezium.snapshot.spi.SnapshotQuery interface. All built-in snapshot query modes implement this interface.

  1. /**
  2. * {@link SnapshotQuery} is used to determine the query used during a data snapshot
  3. *
  4. *
  5. */
  6. public interface SnapshotQuery extends Configurable, Service {
  7. /**
  8. * @return the name of the snapshot lock.
  9. *
  10. *
  11. */
  12. String name();
  13. /**
  14. * Generate a valid query string for the specified table, or an empty {@link Optional}
  15. * to skip snapshotting this table (but that table will still be streamed from)
  16. *
  17. * @param tableId the table to generate a query for
  18. * @param snapshotSelectColumns the columns to be used in the snapshot select based on the column
  19. * include/exclude filters
  20. * @return a valid query string, or none to skip snapshotting this table
  21. */
  22. Optional<String> snapshotQuery(String tableId, List<String> snapshotSelectColumns);
  23. }

io.debezium.snapshot.spi.SnapshotLock interface. All built-in snapshot lock modes implement this interface.

  1. /**
  2. * {@link SnapshotLock} is used to determine the table lock mode used during schema snapshot
  3. *
  4. *
  5. */
  6. public interface SnapshotLock extends Configurable, Service {
  7. /**
  8. * @return the name of the snapshot lock.
  9. *
  10. *
  11. */
  12. String name();
  13. /**
  14. * Returns a SQL statement for locking the given table during snapshotting, if required by the specific snapshotter
  15. * implementation.
  16. */
  17. Optional<String> tableLockingStatement(Duration lockTimeout, String tableId);
  18. }

Blocking snapshots

To provide more flexibility in managing snapshots, Debezium includes a supplementary ad hoc snapshot mechanism, known as a blocking snapshot. Blocking snapshots rely on the Debezium mechanism for sending signals to a Debezium connector.

A blocking snapshot behaves just like an initial snapshot, except that you can trigger it at run time.

You might want to run a blocking snapshot rather than use the standard initial snapshot process in the following situations:

  • You add a new table and you want to complete the snapshot while the connector is running.

  • You add a large table, and you want the snapshot to complete in less time than is possible with an incremental snapshot.

Blocking snapshot process

When you run a blocking snapshot, Debezium stops streaming, and then initiates a snapshot of the specified table, following the same process that it uses during an initial snapshot. After the snapshot completes, the streaming is resumed.

Configure snapshot

You can set the following properties in the data component of a signal:

  • data-collections: to specify which tables must be snapshot

  • additional-conditions: You can specify different filters for different table.

    • The data-collection property is the fully-qualified name of the table for which the filter will be applied.

    • The filter property will have the same value used in the snapshot.select.statement.overrides

For example:

  1. {"type": "blocking", "data-collections": ["schema1.table1", "schema1.table2"], "additional-conditions": [{"data-collection": "schema1.table1", "filter": "SELECT * FROM [schema1].[table1] WHERE column1 = 0 ORDER BY column2 DESC"}, {"data-collection": "schema1.table2", "filter": "SELECT * FROM [schema1].[table2] WHERE column2 > 0"}]}

Possible duplicates

A delay might exist between the time that you send the signal to trigger the snapshot, and the time when streaming stops and the snapshot starts. As a result of this delay, after the snapshot completes, the connector might emit some event records that duplicate records captured by the snapshot.

Reading the change data tables

When the connector first starts, it takes a structural snapshot of the structure of the captured tables and persists this information to its internal database schema history topic. The connector then identifies a change table for each source table, and completes the following steps.

  1. For each change table, the connector read all of the changes that were created between the last stored maximum LSN and the current maximum LSN.

  2. The connector sorts the changes that it reads in ascending order, based on the values of their commit LSN and change LSN. This sorting order ensures that the changes are replayed by Debezium in the same order in which they occurred in the database.

  3. The connector passes the commit and change LSNs as offsets to Kafka Connect.

  4. The connector stores the maximum LSN and restarts the process from Step 1.

After a restart, the connector resumes processing from the last offset (commit and change LSNs) that it read.

The connector is able to detect whether CDC is enabled or disabled for included source tables and adjust its behavior.

No maximum LSN recorded in the database

There may be situations when no maximum LSN is recorded in the database because:

  1. SQL Server Agent is not running

  2. No changes are recorded in the change table yet

  3. Database has low activity and the cdc clean up job periodically clears entries from the cdc tables

Out of these possibilities, since a running SQL Server Agent is a prerequisite, No 1. is a real problem (while No 2. and 3. are normal).

In order to mitigate this issue and differentiate between No 1. and the others, a check for the status of the SQL Server Agent is done through the following query "SELECT CASE WHEN dss.[status]=4 THEN 1 ELSE 0 END AS isRunning FROM [#db].sys.dm_server_services dss WHERE dss.[servicename] LIKE N’SQL Server Agent (%';". If the SQL Server Agent is not running, an ERROR is written in the log: “No maximum LSN recorded in the database; SQL Server Agent is not running”.

The SQL Server Agent running status query requires VIEW SERVER STATE server permission. If you don’t want to grant this permission to the configured user, you can choose to configure your own query through the database.sqlserver.agent.status.query property. You can define a function which returns true or 1 if SQL Server Agent is running (false or 0 otherwise) and safely use High-Level permissions without granting them as explained here What minimum permissions do I need to provide to a user so that it can check the status of SQL Server Agent Service? or here Safely and Easily Use High-Level Permissions Without Granting Them to Anyone: Server-level. The configuration of the query property would look like: database.sqlserver.agent.status.query=SELECT [#db].func_is_sql_server_agent_running() - you need to use [#db] as placeholder for the database name.

Limitations

SQL Server specifically requires the base object to be a table in order to create a change capture instance. As consequence, capturing changes from indexed views (aka. materialized views) is not supported by SQL Server and hence Debezium SQL Server connector.

Topic names

By default, the SQL Server connector writes events for all INSERT, UPDATE, and DELETE operations that occur in a table to a single Apache Kafka topic that is specific to that table. The connector uses the following convention to name change event topics: _<topicPrefix>_._<schemaName>_._<tableName>_

The following list provides definitions for the components of the default name:

topicPrefix

The logical name of the server, as specified by the topic.prefix configuration property.

schemaName

The name of the database schema in which the change event occurred.

tableName

The name of the database table in which the change event occurred.

For example, if fulfillment is the logical server name, and dbo is the schema name, and the database contains tables with the names products, products_on_hand, customers, and orders, the connector would stream change event records to the following Kafka topics:

  • fulfillment.testDB.dbo.products

  • fulfillment.testDB.dbo.products_on_hand

  • fulfillment.testDB.dbo.customers

  • fulfillment.testDB.dbo.orders

The connector applies similar naming conventions to label its internal database schema history topics, schema change topics, and transaction metadata topics.

If the default topic name do not meet your requirements, you can configure custom topic names. To configure custom topic names, you specify regular expressions in the logical topic routing SMT. For more information about using the logical topic routing SMT to customize topic naming, see Topic routing.

Schema history topic

When a database client queries a database, the client uses the database’s current schema. However, the database schema can be changed at any time, which means that the connector must be able to identify what the schema was at the time each insert, update, or delete operation was recorded. Also, a connector cannot necessarily apply the current schema to every event. If an event is relatively old, it’s possible that it was recorded before the current schema was applied.

To ensure correct processing of change events that occur after a schema change, the Debezium SQL Server connector stores a snapshot of the new schema based on the structure in the SQL Server change tables, which mirror the structure of their associated data tables. The connector stores the table schema information, together with the LSN of operations the result in schema changes, in the database schema history Kafka topic. The connector uses the stored schema representation to produce change events that correctly mirror the structure of tables at the time of each insert, update, or delete operation.

When the connector restarts after either a crash or a graceful stop, it resumes reading entries in the SQL Server CDC tables from the last position that it read. Based on the schema information that the connector reads from the database schema history topic, the connector applies the table structures that existed at the position where the connector restarts.

If you update the schema of a Db2 table that is in capture mode, it’s important that you also update the schema of the corresponding change table. You must be a SQL Server database administrator with elevated privileges to update database schema. For more information about updating SQL Server database schema in Debezium environmenbts, see Database schema evolution.

The database schema history topic is for internal connector use only. Optionally, the connector can also emit schema change events to a different topic that is intended for consumer applications.

Additional resources

Schema change topic

For each table for which CDC is enabled, the Debezium SQL Server connector stores a history of the schema change events that are applied to tables in the database. The connector writes schema change events to a Kafka topic named _<topicPrefix>_, where _topicPrefix_ is the logical server name that is specified in the topic.prefix configuration property.

Messages that the connector sends to the schema change topic contain a payload, and, optionally, also contain the schema of the change event message.

The schema for the schema change event has the following elements:

name

The name of the schema change event message.

type

The type of the change event message.

version

The version of the schema. The version is an integer that is incremented each time the schema is changed.

fields

The fields that are included in the change event message.

Example: Schema of the SQL Server connector schema change topic

The following example shows a typical schema in JSON format.

  1. {
  2. "schema": {
  3. "type": "struct",
  4. "fields": [
  5. {
  6. "type": "string",
  7. "optional": false,
  8. "field": "databaseName"
  9. }
  10. ],
  11. "optional": false,
  12. "name": "io.debezium.connector.sqlserver.SchemaChangeKey",
  13. "version": 1
  14. },
  15. "payload": {
  16. "databaseName": "inventory"
  17. }
  18. }

The payload of a schema change event message includes the following elements:

databaseName

The name of the database to which the statements are applied. The value of databaseName serves as the message key.

tableChanges

A structured representation of the entire table schema after the schema change. The tableChanges field contains an array that includes entries for each column of the table. Because the structured representation presents data in JSON or Avro format, consumers can easily read messages without first processing them through a DDL parser.

When the connector is configured to capture a table, it stores the history of the table’s schema changes not only in the schema change topic, but also in an internal database schema history topic. The internal database schema history topic is for connector use only and it is not intended for direct use by consuming applications. Ensure that applications that require notifications about schema changes consume that information only from the schema change topic.

The format of the messages that a connector emits to its schema change topic is in an incubating state and can change without notice.

Debezium emits a message to the schema change topic when the following events occur:

  • You enable CDC for a table.

  • You disable CDC for a table.

  • You alter the structure of a table for which CDC is enabled by following the schema evolution procedure.

Example: Message emitted to the SQL Server connector schema change topic

The following example shows a message in the schema change topic. The message contains a logical representation of the table schema.

  1. {
  2. "schema": {
  3. ...
  4. },
  5. "payload": {
  6. "source": {
  7. "version": "2.6.2.Final",
  8. "connector": "sqlserver",
  9. "name": "server1",
  10. "ts_ms": 0,
  11. "snapshot": "true",
  12. "db": "testDB",
  13. "schema": "dbo",
  14. "table": "customers",
  15. "change_lsn": null,
  16. "commit_lsn": "00000025:00000d98:00a2",
  17. "event_serial_no": null
  18. },
  19. "ts_ms": 1588252618953, (1)
  20. "databaseName": "testDB", (2)
  21. "schemaName": "dbo",
  22. "ddl": null, (3)
  23. "tableChanges": [ (4)
  24. {
  25. "type": "CREATE", (5)
  26. "id": "\"testDB\".\"dbo\".\"customers\"", (6)
  27. "table": { (7)
  28. "defaultCharsetName": null,
  29. "primaryKeyColumnNames": [ (8)
  30. "id"
  31. ],
  32. "columns": [ (9)
  33. {
  34. "name": "id",
  35. "jdbcType": 4,
  36. "nativeType": null,
  37. "typeName": "int identity",
  38. "typeExpression": "int identity",
  39. "charsetName": null,
  40. "length": 10,
  41. "scale": 0,
  42. "position": 1,
  43. "optional": false,
  44. "autoIncremented": false,
  45. "generated": false
  46. },
  47. {
  48. "name": "first_name",
  49. "jdbcType": 12,
  50. "nativeType": null,
  51. "typeName": "varchar",
  52. "typeExpression": "varchar",
  53. "charsetName": null,
  54. "length": 255,
  55. "scale": null,
  56. "position": 2,
  57. "optional": false,
  58. "autoIncremented": false,
  59. "generated": false
  60. },
  61. {
  62. "name": "last_name",
  63. "jdbcType": 12,
  64. "nativeType": null,
  65. "typeName": "varchar",
  66. "typeExpression": "varchar",
  67. "charsetName": null,
  68. "length": 255,
  69. "scale": null,
  70. "position": 3,
  71. "optional": false,
  72. "autoIncremented": false,
  73. "generated": false
  74. },
  75. {
  76. "name": "email",
  77. "jdbcType": 12,
  78. "nativeType": null,
  79. "typeName": "varchar",
  80. "typeExpression": "varchar",
  81. "charsetName": null,
  82. "length": 255,
  83. "scale": null,
  84. "position": 4,
  85. "optional": false,
  86. "autoIncremented": false,
  87. "generated": false
  88. }
  89. ],
  90. "attributes": [ (10)
  91. {
  92. "customAttribute": "attributeValue"
  93. }
  94. ]
  95. }
  96. }
  97. ]
  98. }
  99. }
Table 7. Descriptions of fields in messages emitted to the schema change topic
ItemField nameDescription

1

ts_ms

Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task.

In the source object, ts_ms indicates the time that the change was made in the database. By comparing the value for payload.source.ts_ms with the value for payload.ts_ms, you can determine the lag between the source database update and Debezium.

2

databaseName
schemaName

Identifies the database and the schema that contain the change.

3

ddl

Always null for the SQL Server connector. For other connectors, this field contains the DDL responsible for the schema change. This DDL is not available to SQL Server connectors.

4

tableChanges

An array of one or more items that contain the schema changes generated by a DDL command.

5

type

Describes the kind of change. The value is one of the following:

  • CREATE - table created

  • ALTER - table modified

  • DROP - table deleted

6

id

Full identifier of the table that was created, altered, or dropped.

7

table

Represents table metadata after the applied change.

8

primaryKeyColumnNames

List of columns that compose the table’s primary key.

9

columns

Metadata for each column in the changed table.

10

attributes

Custom attribute metadata for each table change.

In messages that the connector sends to the schema change topic, the key is the name of the database that contains the schema change. In the following example, the payload field contains the key:

  1. {
  2. "schema": {
  3. "type": "struct",
  4. "fields": [
  5. {
  6. "type": "string",
  7. "optional": false,
  8. "field": "databaseName"
  9. }
  10. ],
  11. "optional": false,
  12. "name": "io.debezium.connector.sqlserver.SchemaChangeKey",
  13. "version": 1
  14. },
  15. "payload": {
  16. "databaseName": "testDB"
  17. }
  18. }

Data change events

The Debezium SQL Server connector generates a data change event for each row-level INSERT, UPDATE, and DELETE operation. Each event contains a key and a value. The structure of the key and the value depends on the table that was changed.

Debezium and Kafka Connect are designed around continuous streams of event messages. However, the structure of these events may change over time, which can be difficult for consumers to handle. To address this, each event contains the schema for its content or, if you are using a schema registry, a schema ID that a consumer can use to obtain the schema from the registry. This makes each event self-contained.

The following skeleton JSON shows the basic four parts of a change event. However, how you configure the Kafka Connect converter that you choose to use in your application determines the representation of these four parts in change events. A schema field is in a change event only when you configure the converter to produce it. Likewise, the event key and event payload are in a change event only if you configure a converter to produce it. If you use the JSON converter and you configure it to produce all four basic change event parts, change events have this structure:

  1. {
  2. "schema": { (1)
  3. ...
  4. },
  5. "payload": { (2)
  6. ...
  7. },
  8. "schema": { (3)
  9. ...
  10. },
  11. "payload": { (4)
  12. ...
  13. },
  14. }
Table 8. Overview of change event basic content
ItemField nameDescription

1

schema

The first schema field is part of the event key. It specifies a Kafka Connect schema that describes what is in the event key’s payload portion. In other words, the first schema field describes the structure of the primary key, or the unique key if the table does not have a primary key, for the table that was changed.

It is possible to override the table’s primary key by setting the message.key.columns connector configuration property. In this case, the first schema field describes the structure of the key identified by that property.

2

payload

The first payload field is part of the event key. It has the structure described by the previous schema field and it contains the key for the row that was changed.

3

schema

The second schema field is part of the event value. It specifies the Kafka Connect schema that describes what is in the event value’s payload portion. In other words, the second schema describes the structure of the row that was changed. Typically, this schema contains nested schemas.

4

payload

The second payload field is part of the event value. It has the structure described by the previous schema field and it contains the actual data for the row that was changed.

By default, the connector streams change event records to topics with names that are the same as the event’s originating table. For more information, see topic names.

The SQL Server connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This means that the logical server name must start with a Latin letter or an underscore, that is, a-z, A-Z, or . Each remaining character in the logical server name and each character in the database and table names must be a Latin letter, a digit, or an underscore, that is, a-z, A-Z, 0-9, or \. If there is an invalid character it is replaced with an underscore character.

This can lead to unexpected conflicts if the logical server name, a database name, or a table name contains invalid characters, and the only characters that distinguish names from one another are invalid and thus replaced with underscores.

Change event keys

A change event’s key contains the schema for the changed table’s key and the changed row’s actual key. Both the schema and its corresponding payload contain a field for each column in the changed table’s primary key (or unique key constraint) at the time the connector created the event.

Consider the following customers table, which is followed by an example of a change event key for this table.

Example table

  1. CREATE TABLE customers (
  2. id INTEGER IDENTITY(1001,1) NOT NULL PRIMARY KEY,
  3. first_name VARCHAR(255) NOT NULL,
  4. last_name VARCHAR(255) NOT NULL,
  5. email VARCHAR(255) NOT NULL UNIQUE
  6. );

Example change event key

Every change event that captures a change to the customers table has the same event key schema. For as long as the customers table has the previous definition, every change event that captures a change to the customers table has the following key structure, which in JSON, looks like this:

  1. {
  2. "schema": { (1)
  3. "type": "struct",
  4. "fields": [ (2)
  5. {
  6. "type": "int32",
  7. "optional": false,
  8. "field": "id"
  9. }
  10. ],
  11. "optional": false, (3)
  12. "name": "server1.testDB.dbo.customers.Key" (4)
  13. },
  14. "payload": { (5)
  15. "id": 1004
  16. }
  17. }
Table 9. Description of change event key
ItemField nameDescription

1

schema

The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s payload portion.

2

fields

Specifies each field that is expected in the payload, including each field’s name, type, and whether it is required. In this example, there is one required field named id of type int32.

3

optional

Indicates whether the event key must contain a value in its payload field. In this example, a value in the key’s payload is required. A value in the key’s payload field is optional when a table does not have a primary key.

4

server1.dbo.testDB.customers.Key

Name of the schema that defines the structure of the key’s payload. This schema describes the structure of the primary key for the table that was changed. Key schema names have the format connector-name.database-schema-name.table-name.Key. In this example:

  • server1 is the name of the connector that generated this event.

  • dbo is the database schema for the table that was changed.

  • customers is the table that was updated.

5

payload

Contains the key for the row for which this change event was generated. In this example, the key, contains a single id field whose value is 1004.

When Debezium emits a change event record, it sets the message key for each record to the name of the primary key or unique key column of the source table. Debezium must be able to read these columns to function properly. If you set the column.include.list or column.exclude.list properties in the connector configuration, be sure that your settings permit the connector to capture the required primary key or unique key columns.

If the table does not have a primary or unique key, then the change event’s key is null. This makes sense since the rows in a table without a primary or unique key constraint cannot be uniquely identified.

Change event values

The value in a change event is a bit more complicated than the key. Like the key, the value has a schema section and a payload section. The schema section contains the schema that describes the Envelope structure of the payload section, including its nested fields. Change events for operations that create, update or delete data all have a value payload with an envelope structure.

Consider the same sample table that was used to show an example of a change event key:

  1. CREATE TABLE customers (
  2. id INTEGER IDENTITY(1001,1) NOT NULL PRIMARY KEY,
  3. first_name VARCHAR(255) NOT NULL,
  4. last_name VARCHAR(255) NOT NULL,
  5. email VARCHAR(255) NOT NULL UNIQUE
  6. );

The value portion of a change event for a change to this table is described for each event type.

create events

The following example shows the value portion of a change event that the connector generates for an operation that creates data in the customers table:

  1. {
  2. "schema": { (1)
  3. "type": "struct",
  4. "fields": [
  5. {
  6. "type": "struct",
  7. "fields": [
  8. {
  9. "type": "int32",
  10. "optional": false,
  11. "field": "id"
  12. },
  13. {
  14. "type": "string",
  15. "optional": false,
  16. "field": "first_name"
  17. },
  18. {
  19. "type": "string",
  20. "optional": false,
  21. "field": "last_name"
  22. },
  23. {
  24. "type": "string",
  25. "optional": false,
  26. "field": "email"
  27. }
  28. ],
  29. "optional": true,
  30. "name": "server1.dbo.testDB.customers.Value", (2)
  31. "field": "before"
  32. },
  33. {
  34. "type": "struct",
  35. "fields": [
  36. {
  37. "type": "int32",
  38. "optional": false,
  39. "field": "id"
  40. },
  41. {
  42. "type": "string",
  43. "optional": false,
  44. "field": "first_name"
  45. },
  46. {
  47. "type": "string",
  48. "optional": false,
  49. "field": "last_name"
  50. },
  51. {
  52. "type": "string",
  53. "optional": false,
  54. "field": "email"
  55. }
  56. ],
  57. "optional": true,
  58. "name": "server1.dbo.testDB.customers.Value",
  59. "field": "after"
  60. },
  61. {
  62. "type": "struct",
  63. "fields": [
  64. {
  65. "type": "string",
  66. "optional": false,
  67. "field": "version"
  68. },
  69. {
  70. "type": "string",
  71. "optional": false,
  72. "field": "connector"
  73. },
  74. {
  75. "type": "string",
  76. "optional": false,
  77. "field": "name"
  78. },
  79. {
  80. "type": "int64",
  81. "optional": false,
  82. "field": "ts_ms"
  83. },
  84. {
  85. "type": "int64",
  86. "optional": false,
  87. "field": "ts_us"
  88. },
  89. {
  90. "type": "int64",
  91. "optional": false,
  92. "field": "ts_ns"
  93. },
  94. {
  95. "type": "boolean",
  96. "optional": true,
  97. "default": false,
  98. "field": "snapshot"
  99. },
  100. {
  101. "type": "string",
  102. "optional": false,
  103. "field": "db"
  104. },
  105. {
  106. "type": "string",
  107. "optional": false,
  108. "field": "schema"
  109. },
  110. {
  111. "type": "string",
  112. "optional": false,
  113. "field": "table"
  114. },
  115. {
  116. "type": "string",
  117. "optional": true,
  118. "field": "change_lsn"
  119. },
  120. {
  121. "type": "string",
  122. "optional": true,
  123. "field": "commit_lsn"
  124. },
  125. {
  126. "type": "int64",
  127. "optional": true,
  128. "field": "event_serial_no"
  129. }
  130. ],
  131. "optional": false,
  132. "name": "io.debezium.connector.sqlserver.Source", (3)
  133. "field": "source"
  134. },
  135. {
  136. "type": "string",
  137. "optional": false,
  138. "field": "op"
  139. },
  140. {
  141. "type": "int64",
  142. "optional": true,
  143. "field": "ts_ms"
  144. },
  145. {
  146. "type": "int64",
  147. "optional": true,
  148. "field": "ts_us"
  149. },
  150. {
  151. "type": "int64",
  152. "optional": true,
  153. "field": "ts_ns"
  154. }
  155. ],
  156. "optional": false,
  157. "name": "server1.dbo.testDB.customers.Envelope" (4)
  158. },
  159. "payload": { (5)
  160. "before": null, (6)
  161. "after": { (7)
  162. "id": 1005,
  163. "first_name": "john",
  164. "last_name": "doe",
  165. "email": "john.doe@example.org"
  166. },
  167. "source": { (8)
  168. "version": "2.6.2.Final",
  169. "connector": "sqlserver",
  170. "name": "server1",
  171. "ts_ms": 1559729468470,
  172. "ts_us": 1559729468470000,
  173. "ts_ns": 1559729468470000000,
  174. "snapshot": false,
  175. "db": "testDB",
  176. "schema": "dbo",
  177. "table": "customers",
  178. "change_lsn": "00000027:00000758:0003",
  179. "commit_lsn": "00000027:00000758:0005",
  180. "event_serial_no": "1"
  181. },
  182. "op": "c", (9)
  183. "ts_ms": 1559729471739, (10)
  184. "ts_ms": 1559729471739876, (10)
  185. "ts_ms": 1559729471739876149 (10)
  186. }
  187. }
Table 10. Descriptions of create event value fields
ItemField nameDescription

1

schema

The value’s schema, which describes the structure of the value’s payload. A change event’s value schema is the same in every change event that the connector generates for a particular table.

2

name

In the schema section, each name field specifies the schema for a field in the value’s payload.

server1.dbo.testDB.customers.Value is the schema for the payload’s before and after fields. This schema is specific to the customers table.

Names of schemas for before and after fields are of the form logicalName.database-schemaName.tableName.Value, which ensures that the schema name is unique in the database. This means that when using the Avro converter, the resulting Avro schema for each table in each logical source has its own evolution and history.

3

name

io.debezium.connector.sqlserver.Source is the schema for the payload’s source field. This schema is specific to the SQL Server connector. The connector uses it for all events that it generates.

4

name

server1.dbo.testDB.customers.Envelope is the schema for the overall structure of the payload, where server1 is the connector name, dbo is the database schema name, and customers is the table.

5

payload

The value’s actual data. This is the information that the change event is providing.

It may appear that the JSON representations of the events are much larger than the rows they describe. This is because the JSON representation must include the schema and the payload portions of the message. However, by using the Avro converter, you can significantly decrease the size of the messages that the connector streams to Kafka topics.

6

before

An optional field that specifies the state of the row before the event occurred. When the op field is c for create, as it is in this example, the before field is null since this change event is for new content.

7

after

An optional field that specifies the state of the row after the event occurred. In this example, the after field contains the values of the new row’s id, first_name, last_name, and email columns.

8

source

Mandatory field that describes the source metadata for the event. This field contains information that you can use to compare this event with other events, with regard to the origin of the events, the order in which the events occurred, and whether events were part of the same transaction. The source metadata includes:

  • Debezium version

  • Connector type and name

  • Database and schema names

  • Timestamp for when the change was made in the database

  • If the event was part of a snapshot

  • Name of the table that contains the new row

  • Server log offsets

9

op

Mandatory string that describes the type of operation that caused the connector to generate the event. In this example, c indicates that the operation created a row. Valid values are:

  • c = create

  • u = update

  • d = delete

  • r = read (applies to only snapshots)

10

ts_ms, ts_us, ts_ns

Optional field that displays the time at which the connector processed the event. In the event message envelope, the time is based on the system clock in the JVM running the Kafka Connect task.

In the source object, ts_ms indicates the time when a change was committed in the database. By comparing the value for payload.source.ts_ms with the value for payload.ts_ms, you can determine the lag between the source database update and Debezium.

update events

The value of a change event for an update in the sample customers table has the same schema as a create event for that table. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in an update event. Here is an example of a change event value in an event that the connector generates for an update in the customers table:

  1. {
  2. "schema": { ... },
  3. "payload": {
  4. "before": { (1)
  5. "id": 1005,
  6. "first_name": "john",
  7. "last_name": "doe",
  8. "email": "john.doe@example.org"
  9. },
  10. "after": { (2)
  11. "id": 1005,
  12. "first_name": "john",
  13. "last_name": "doe",
  14. "email": "noreply@example.org"
  15. },
  16. "source": { (3)
  17. "version": "2.6.2.Final",
  18. "connector": "sqlserver",
  19. "name": "server1",
  20. "ts_ms": 1559729995937,
  21. "ts_us": 1559729995937000,
  22. "ts_ns": 1559729995937000000,
  23. "snapshot": false,
  24. "db": "testDB",
  25. "schema": "dbo",
  26. "table": "customers",
  27. "change_lsn": "00000027:00000ac0:0002",
  28. "commit_lsn": "00000027:00000ac0:0007",
  29. "event_serial_no": "2"
  30. },
  31. "op": "u", (4)
  32. "ts_ms": 1559729998706, (5)
  33. "ts_us": 1559729998706318, (5)
  34. "ts_ns": 1559729998706318547 (5)
  35. }
  36. }
Table 11. Descriptions of update event value fields
ItemField nameDescription

1

before

An optional field that specifies the state of the row before the event occurred. In an update event value, the before field contains a field for each table column and the value that was in that column before the database commit. In this example, the email value is john.doe@example.org.

2

after

An optional field that specifies the state of the row after the event occurred. You can compare the before and after structures to determine what the update to this row was. In the example, the email value is now noreply@example.org.

3

source

Mandatory field that describes the source metadata for the event. The source field structure has the same fields as in a create event, but some values are different, for example, the sample update event has a different offset. The source metadata includes:

  • Debezium version

  • Connector type and name

  • Database and schema names

  • Timestamp for when the change was made in the database

  • If the event was part of a snapshot

  • Name of the table that contains the new row

  • Server log offsets

The event_serial_no field differentiates events that have the same commit and change LSN. Typical situations for when this field has a value other than 1:

  • update events have the value set to 2 because the update generates two events in the CDC change table of SQL Server (see the source documentation for details). The first event contains the old values and the second contains contains new values. The connector uses values in the first event to create the second event. The connector drops the first event.

  • When a primary key is updated SQL Server emits two events. A delete event for the removal of the record with the old primary key value and a create event for the addition of the record with the new primary key. Both operations share the same commit and change LSN and their event numbers are 1 and 2, respectively.

4

op

Mandatory string that describes the type of operation. In an update event value, the op field value is u, signifying that this row changed because of an update.

5

ts_ms, ts_us, ts_ns

Optional field that displays the time at which the connector processed the event. In the event message envelope, the time is based on the system clock in the JVM running the Kafka Connect task.

In the source object, ts_ms indicates the time when the change was committed to the database. By comparing the value for payload.source.ts_ms with the value for payload.ts_ms, you can determine the lag between the source database update and Debezium.

Updating the columns for a row’s primary/unique key changes the value of the row’s key. When a key changes, Debezium outputs three events: a delete event and a tombstone event with the old key for the row, followed by a create event with the new key for the row.

delete events

The value in a delete change event has the same schema portion as create and update events for the same table. The payload portion in a delete event for the sample customers table looks like this:

  1. {
  2. "schema": { ... },
  3. },
  4. "payload": {
  5. "before": { <>
  6. "id": 1005,
  7. "first_name": "john",
  8. "last_name": "doe",
  9. "email": "noreply@example.org"
  10. },
  11. "after": null, (2)
  12. "source": { (3)
  13. "version": "2.6.2.Final",
  14. "connector": "sqlserver",
  15. "name": "server1",
  16. "ts_ms": 1559730445243,
  17. "ts_us": 1559730445243000,
  18. "ts_ns": 1559730445243000000,
  19. "snapshot": false,
  20. "db": "testDB",
  21. "schema": "dbo",
  22. "table": "customers",
  23. "change_lsn": "00000027:00000db0:0005",
  24. "commit_lsn": "00000027:00000db0:0007",
  25. "event_serial_no": "1"
  26. },
  27. "op": "d", (4)
  28. "ts_ms": 1559730450205, (5)
  29. "ts_us": 1559730450205387, (5)
  30. "ts_ns": 1559730450205387492 (5)
  31. }
  32. }
Table 12. Descriptions of delete event value fields
ItemField nameDescription

1

before

Optional field that specifies the state of the row before the event occurred. In a delete event value, the before field contains the values that were in the row before it was deleted with the database commit.

2

after

Optional field that specifies the state of the row after the event occurred. In a delete event value, the after field is null, signifying that the row no longer exists.

3

source

Mandatory field that describes the source metadata for the event. In a delete event value, the source field structure is the same as for create and update events for the same table. Many source field values are also the same. In a delete event value, the ts_ms and pos field values, as well as other values, might have changed. But the source field in a delete event value provides the same metadata:

  • Debezium version

  • Connector type and name

  • Database and schema names

  • Timestamp for when the change was made in the database

  • If the event was part of a snapshot

  • Name of the table that contains the new row

  • Server log offsets

4

op

Mandatory string that describes the type of operation. The op field value is d, signifying that this row was deleted.

5

ts_ms, ts_us, ts_ns

Optional field that displays the time at which the connector processed the event. In the event message envelope, the time is based on the system clock in the JVM running the Kafka Connect task.

In the source object, ts_ms indicates the time that the change was made in the database. By comparing the value for payload.source.ts_ms with the value for payload.ts_ms, you can determine the lag between the source database update and Debezium.

SQL Server connector events are designed to work with Kafka log compaction. Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.

Tombstone events

When a row is deleted, the delete event value still works with log compaction, because Kafka can remove all earlier messages that have that same key. However, for Kafka to remove all messages that have that same key, the message value must be null. To make this possible, after Debezium’s SQL Server connector emits a delete event, the connector emits a special tombstone event that has the same key but a null value.

Transaction metadata

Debezium can generate events that represent transaction boundaries and that enrich data change event messages.

Limits on when Debezium receives transaction metadata

Debezium registers and receives metadata only for transactions that occur after you deploy the connector. Metadata for transactions that occur before you deploy the connector is not available.

Database transactions are represented by a statement block that is enclosed between the BEGIN and END keywords. Debezium generates transaction boundary events for the BEGIN and END delimiters in every transaction. Transaction boundary events contain the following fields:

status

BEGIN or END.

id

String representation of the unique transaction identifier.

ts_ms

The time of a transaction boundary event (BEGIN or END event) at the data source. If the data source does not provide Debezium with the event time, then the field instead represents the time at which Debezium processes the event.

event_count (for END events)

Total number of events emmitted by the transaction.

data_collections (for END events)

An array of pairs of data_collection and event_count elements that indicates the number of events that the connector emits for changes that originate from a data collection.

There is no way for Debezium to reliably identify when a transaction has ended. The transaction END marker is thus emitted only after the first event of another transaction arrives. This can lead to the delayed delivery of END marker in case of a low-traffic system.

The following example shows a typical transaction boundary message:

Example: SQL Server connector transaction boundary event

  1. {
  2. "status": "BEGIN",
  3. "id": "00000025:00000d08:0025",
  4. "ts_ms": 1486500577125,
  5. "event_count": null,
  6. "data_collections": null
  7. }
  8. {
  9. "status": "END",
  10. "id": "00000025:00000d08:0025",
  11. "ts_ms": 1486500577691,
  12. "event_count": 2,
  13. "data_collections": [
  14. {
  15. "data_collection": "testDB.dbo.testDB.tablea",
  16. "event_count": 1
  17. },
  18. {
  19. "data_collection": "testDB.dbo.testDB.tableb",
  20. "event_count": 1
  21. }
  22. ]
  23. }

Unless overridden via the topic.transaction option, transaction events are written to the topic named .transaction.

Change data event enrichment

When transaction metadata is enabled, the data message Envelope is enriched with a new transaction field. This field provides information about every event in the form of a composite of fields:

id

String representation of unique transaction identifier

total_order

The absolute position of the event among all events generated by the transaction

data_collection_order

The per-data collection position of the event among all events that were emitted by the transaction

The following example shows what a typical message looks like:

  1. {
  2. "before": null,
  3. "after": {
  4. "pk": "2",
  5. "aa": "1"
  6. },
  7. "source": {
  8. ...
  9. },
  10. "op": "c",
  11. "ts_ms": "1580390884335",
  12. "ts_us": "1580390884335172",
  13. "ts_ns": "1580390884335172574",
  14. "transaction": {
  15. "id": "00000025:00000d08:0025",
  16. "total_order": "1",
  17. "data_collection_order": "1"
  18. }
  19. }

Data type mappings

The Debezium SQL Server connector represents changes to table row data by producing events that are structured like the table in which the row exists. Each event contains fields to represent the column values for the row. The way in which an event represents the column values for an operation depends on the SQL data type of the column. In the event, the connector maps the fields for each SQL Server data type to both a literal type and a semantic type.

The connector can map SQL Server data types to both literal and semantic types.

Literal type

Describes how the value is literally represented by using Kafka Connect schema types, namely INT8, INT16, INT32, INT64, FLOAT32, FLOAT64, BOOLEAN, STRING, BYTES, ARRAY, MAP, and STRUCT.

Semantic type

Describes how the Kafka Connect schema captures the meaning of the field using the name of the Kafka Connect schema for the field.

If the default data type conversions do not meet your needs, you can create a custom converter for the connector.

Basic types

The following table shows how the connector maps basic SQL Server data types.

Table 13. Data type mappings used by the SQL Server connector
SQL Server data typeLiteral type (schema type)Semantic type (schema name) and Notes

BIT

BOOLEAN

n/a

TINYINT

INT16

n/a

SMALLINT

INT16

n/a

INT

INT32

n/a

BIGINT

INT64

n/a

REAL

FLOAT32

n/a

FLOAT[(N)]

FLOAT64

n/a

CHAR[(N)]

STRING

n/a

VARCHAR[(N)]

STRING

n/a

TEXT

STRING

n/a

NCHAR[(N)]

STRING

n/a

NVARCHAR[(N)]

STRING

n/a

NTEXT

STRING

n/a

XML

STRING

io.debezium.data.Xml

Contains the string representation of an XML document

DATETIMEOFFSET[(P)]

STRING

io.debezium.time.ZonedTimestamp

A string representation of a timestamp with timezone information, where the timezone is GMT

Other data type mappings are described in the following sections.

If present, a column’s default value is propagated to the corresponding field’s Kafka Connect schema. Change messages will contain the field’s default value (unless an explicit column value had been given), so there should rarely be the need to obtain the default value from the schema. Passing the default value helps though with satisfying the compatibility rules when using Avro as serialization format together with the Confluent schema registry.

Temporal values

Other than SQL Server’s DATETIMEOFFSET data type (which contain time zone information), the other temporal types depend on the value of the time.precision.mode configuration property. When the time.precision.mode configuration property is set to adaptive (the default), then the connector will determine the literal type and semantic type for the temporal types based on the column’s data type definition so that events exactly represent the values in the database:

SQL Server data typeLiteral type (schema type)Semantic type (schema name) and Notes

DATE

INT32

io.debezium.time.Date

Represents the number of days since the epoch.

TIME(0), TIME(1), TIME(2), TIME(3)

INT32

io.debezium.time.Time

Represents the number of milliseconds past midnight, and does not include timezone information.

TIME(4), TIME(5), TIME(6)

INT64

io.debezium.time.MicroTime

Represents the number of microseconds past midnight, and does not include timezone information.

TIME(7)

INT64

io.debezium.time.NanoTime

Represents the number of nanoseconds past midnight, and does not include timezone information.

DATETIME

INT64

io.debezium.time.Timestamp

Represents the number of milliseconds past the epoch, and does not include timezone information.

SMALLDATETIME

INT64

io.debezium.time.Timestamp

Represents the number of milliseconds past the epoch, and does not include timezone information.

DATETIME2(0), DATETIME2(1), DATETIME2(2), DATETIME2(3)

INT64

io.debezium.time.Timestamp

Represents the number of milliseconds past the epoch, and does not include timezone information.

DATETIME2(4), DATETIME2(5), DATETIME2(6)

INT64

io.debezium.time.MicroTimestamp

Represents the number of microseconds past the epoch, and does not include timezone information.

DATETIME2(7)

INT64

io.debezium.time.NanoTimestamp

Represents the number of nanoseconds past the epoch, and does not include timezone information.

When the time.precision.mode configuration property is set to connect, then the connector will use the predefined Kafka Connect logical types. This may be useful when consumers only know about the built-in Kafka Connect logical types and are unable to handle variable-precision time values. On the other hand, since SQL Server supports tenth of microsecond precision, the events generated by a connector with the connect time precision mode will result in a loss of precision when the database column has a fractional second precision value greater than 3:

SQL Server data typeLiteral type (schema type)Semantic type (schema name) and Notes

DATE

INT32

org.apache.kafka.connect.data.Date

Represents the number of days since the epoch.

TIME([P])

INT64

org.apache.kafka.connect.data.Time

Represents the number of milliseconds since midnight, and does not include timezone information. SQL Server allows P to be in the range 0-7 to store up to tenth of a microsecond precision, though this mode results in a loss of precision when P > 3.

DATETIME

INT64

org.apache.kafka.connect.data.Timestamp

Represents the number of milliseconds since the epoch, and does not include timezone information.

SMALLDATETIME

INT64

org.apache.kafka.connect.data.Timestamp

Represents the number of milliseconds past the epoch, and does not include timezone information.

DATETIME2

INT64

org.apache.kafka.connect.data.Timestamp

Represents the number of milliseconds since the epoch, and does not include timezone information. SQL Server allows P to be in the range 0-7 to store up to tenth of a microsecond precision, though this mode results in a loss of precision when P > 3.

Timestamp values

The DATETIME, SMALLDATETIME and DATETIME2 types represent a timestamp without time zone information. Such columns are converted into an equivalent Kafka Connect value based on UTC. So for instance the DATETIME2 value “2018-06-20 15:13:16.945104” is represented by a io.debezium.time.MicroTimestamp with the value “1529507596945104”.

Note that the timezone of the JVM running Kafka Connect and Debezium does not affect this conversion.

Decimal values

Debezium connectors handle decimals according to the setting of the decimal.handling.mode connector configuration property.

decimal.handling.mode=precise

Table 14. Mappings when decimal.handling.mode=precise
SQL Server typeLiteral type (schema type)Semantic type (schema name)

NUMERIC[(P[,S])]

BYTES

org.apache.kafka.connect.data.Decimal
The scale schema parameter contains an integer that represents how many digits the decimal point shifted.

DECIMAL[(P[,S])]

BYTES

org.apache.kafka.connect.data.Decimal
The scale schema parameter contains an integer that represents how many digits the decimal point shifted.

SMALLMONEY

BYTES

org.apache.kafka.connect.data.Decimal
The scale schema parameter contains an integer that represents how many digits the decimal point shifted.

MONEY

BYTES

org.apache.kafka.connect.data.Decimal
The scale schema parameter contains an integer that represents how many digits the decimal point shifted.

decimal.handling.mode=double

Table 15. Mappings when decimal.handling.mode=double
SQL Server typeLiteral typeSemantic type

NUMERIC[(M[,D])]

FLOAT64

n/a

DECIMAL[(M[,D])]

FLOAT64

n/a

SMALLMONEY[(M[,D])]

FLOAT64

n/a

MONEY[(M[,D])]

FLOAT64

n/a

decimal.handling.mode=string

Table 16. Mappings when decimal.handling.mode=string
SQL Server typeLiteral typeSemantic type

NUMERIC[(M[,D])]

STRING

n/a

DECIMAL[(M[,D])]

STRING

n/a

SMALLMONEY[(M[,D])]

STRING

n/a

MONEY[(M[,D])]

STRING

n/a

Setting up SQL Server

For Debezium to capture change events from SQL Server tables, a SQL Server administrator with the necessary privileges must first run a query to enable CDC on the database. The administrator must then enable CDC for each table that you want Debezium to capture.

By default, JDBC connections to Microsoft SQL Server are protected by SSL encryption. If SSL is not enabled for a SQL Server database, or if you want to connect to the database without using SSL, you can disable SSL by setting the value of the database.encrypt property in connector configuration to false.

After CDC is applied, it captures all of the INSERT, UPDATE, and DELETE operations that are committed to the tables for which CDC is enabled. The Debezium connector can then capture these events and emit them to Kafka topics.

Enabling CDC on the SQL Server database

Before you can enable CDC for a table, you must enable it for the SQL Server database. A SQL Server administrator enables CDC by running a system stored procedure. System stored procedures can be run by using SQL Server Management Studio, or by using Transact-SQL.

Prerequisites

  • You are a member of the sysadmin fixed server role for the SQL Server.

  • You are a db_owner of the database.

  • The SQL Server Agent is running.

The SQL Server CDC feature processes changes that occur in user-created tables only. You cannot enable CDC on the SQL Server master database.

Procedure

  1. From the View menu in SQL Server Management Studio, click Template Explorer.

  2. In the Template Browser, expand SQL Server Templates.

  3. Expand Change Data Capture > Configuration and then click Enable Database for CDC.

  4. In the template, replace the database name in the USE statement with the name of the database that you want to enable for CDC.

  5. Run the stored procedure sys.sp_cdc_enable_db to enable the database for CDC.

    After the database is enabled for CDC, a schema with the name cdc is created, along with a CDC user, metadata tables, and other system objects.

    The following example shows how to enable CDC for the database MyDB:

    Example: Enabling a SQL Server database for the CDC template

    1. USE MyDB
    2. GO
    3. EXEC sys.sp_cdc_enable_db
    4. GO

Enabling CDC on a SQL Server table

A SQL Server administrator must enable change data capture on the source tables that you want to Debezium to capture. The database must already be enabled for CDC. To enable CDC on a table, a SQL Server administrator runs the stored procedure sys.sp_cdc_enable_table for the table. The stored procedures can be run by using SQL Server Management Studio, or by using Transact-SQL. SQL Server CDC must be enabled for every table that you want to capture.

Prerequisites

  • CDC is enabled on the SQL Server database.

  • The SQL Server Agent is running.

  • You are a member of the db_owner fixed database role for the database.

Procedure

  1. From the View menu in SQL Server Management Studio, click Template Explorer.

  2. In the Template Browser, expand SQL Server Templates.

  3. Expand Change Data Capture > Configuration, and then click Enable Table Specifying Filegroup Option.

  4. In the template, replace the table name in the USE statement with the name of the table that you want to capture.

  5. Run the stored procedure sys.sp_cdc_enable_table.

    The following example shows how to enable CDC for the table MyTable:

    Example: Enabling CDC for a SQL Server table

    1. USE MyDB
    2. GO
    3. EXEC sys.sp_cdc_enable_table
    4. @source_schema = N'dbo',
    5. @source_name = N'MyTable', (1)
    6. @role_name = N'MyRole', (2)
    7. @filegroup_name = N'MyDB_CT',(3)
    8. @supports_net_changes = 0
    9. GO
    1Specifies the name of the table that you want to capture.
    2Specifies a role MyRole to which you can add users to whom you want to grant SELECT permission on the captured columns of the source table. Users in the sysadmin or db_owner role also have access to the specified change tables. Set the value of @role_name to NULL, to allow only members in the sysadmin or db_owner to have full access to captured information.
    3Specifies the filegroup where SQL Server places the change table for the captured table. The named filegroup must already exist. It is best not to locate change tables in the same filegroup that you use for source tables.

Verifying that the user has access to the CDC table

A SQL Server administrator can run a system stored procedure to query a database or table to retrieve its CDC configuration information. The stored procedures can be run by using SQL Server Management Studio, or by using Transact-SQL.

Prerequisites

  • You have SELECT permission on all of the captured columns of the capture instance. Members of the db_owner database role can view information for all of the defined capture instances.

  • You have membership in any gating roles that are defined for the table information that the query includes.

Procedure

  1. From the View menu in SQL Server Management Studio, click Object Explorer.

  2. From the Object Explorer, expand Databases, and then expand your database object, for example, MyDB.

  3. Expand Programmability > Stored Procedures > System Stored Procedures.

  4. Run the sys.sp_cdc_help_change_data_capture stored procedure to query the table.

    Queries should not return empty results.

    The following example runs the stored procedure sys.sp_cdc_help_change_data_capture on the database MyDB:

    Example: Querying a table for CDC configuration information

    1. USE MyDB;
    2. GO
    3. EXEC sys.sp_cdc_help_change_data_capture
    4. GO

    The query returns configuration information for each table in the database that is enabled for CDC and that contains change data that the caller is authorized to access. If the result is empty, verify that the user has privileges to access both the capture instance and the CDC tables.

SQL Server on Azure

The Debezium SQL Server connector can be used with SQL Server on Azure. Refer to this example for configuring CDC for SQL Server on Azure and using it with Debezium.

SQL Server Always On

The SQL Server connector can capture changes from an Always On read-only replica.

Prerequisites

  • Change data capture is configured and enabled on the primary node. SQL Server does not support CDC directly on replicas.

  • The configuration option database.applicationIntent is set to ReadOnly. This is required by SQL Server. When Debezium detects this configuration option, it responds by taking the following actions:

    • Sets snapshot.isolation.mode to snapshot, which is the only one transaction isolation mode supported for read-only replicas.

    • Commits the (read-only) transaction in every execution of the streaming query loop, which is necessary to get the latest view of CDC data.

Effect of SQL Server capture job agent configuration on server load and latency

When a database administrator enables change data capture for a source table, the capture job agent begins to run. The agent reads new change event records from the transaction log and replicates the event records to a change data table. Between the time that a change is committed in the source table, and the time that the change appears in the corresponding change table, there is always a small latency interval. This latency interval represents a gap between when changes occur in the source table and when they become available for Debezium to stream to Apache Kafka.

Ideally, for applications that must respond quickly to changes in data, you want to maintain close synchronization between the source and change tables. You might imagine that running the capture agent to continuously process change events as rapidly as possible might result in increased throughput and reduced latency — populating change tables with new event records as soon as possible after the events occur, in near real time. However, this is not necessarily the case. There is a performance penalty to pay in the pursuit of more immediate synchronization. Each time that the capture job agent queries the database for new event records, it increases the CPU load on the database host. The additional load on the server can have a negative effect on overall database performance, and potentially reduce transaction efficiency, especially during times of peak database use.

It’s important to monitor database metrics so that you know if the database reaches the point where the server can no longer support the capture agent’s level of activity. If you notice performance problems, there are SQL Server capture agent settings that you can modify to help balance the overall CPU load on the database host with a tolerable degree of latency.

SQL Server capture job agent configuration parameters

On SQL Server, parameters that control the behavior of the capture job agent are defined in the SQL Server table msdb.dbo.cdc_jobs. If you experience performance issues while running the capture job agent, adjust capture jobs settings to reduce CPU load by running the sys.sp_cdc_change_job stored procedure and supplying new values.

Specific guidance about how to configure SQL Server capture job agent parameters is beyond the scope of this documentation.

The following parameters are the most significant for modifying capture agent behavior for use with the Debezium SQL Server connector:

pollinginterval

  • Specifies the number of seconds that the capture agent waits between log scan cycles.

  • A higher value reduces the load on the database host and increases latency.

  • A value of 0 specifies no wait between scans.

  • The default value is 5.

maxtrans

  • Specifies the maximum number of transactions to process during each log scan cycle. After the capture job processes the specified number of transactions, it pauses for the length of time that the pollinginterval specifies before the next scan begins.

  • A lower value reduces the load on the database host and increases latency.

  • The default value is 500.

maxscans

  • Specifies a limit on the number of scan cycles that the capture job can attempt in capturing the full contents of the database transaction log. If the continuous parameter is set to 1, the job pauses for the length of time that the pollinginterval specifies before it resumes scanning.

  • A lower values reduces the load on the database host and increases latency.

  • The default value is 10.

Additional resources

  • For more information about capture agent parameters, see the SQL Server documentation.

Deployment

To deploy a Debezium SQL Server connector, you install the Debezium SQL Server connector archive, configure the connector, and start the connector by adding its configuration to Kafka Connect.

Prerequisites

Procedure

  1. Download the Debezium SQL Server connector plug-in archive

  2. Extract the files into your Kafka Connect environment.

  3. Add the directory with the JAR files to Kafka Connect’s plugin.path.

  4. Configure the connector and add the configuration to your Kafka Connect cluster.

  5. Restart your Kafka Connect process to pick up the new JAR files.

If you are working with immutable containers, see Debezium’s container images for Åpache ZooKeeper, Apache Kafka, and Kafka Connect. You can pull the official container images for Microsoft SQL Server on Linux from Docker Hub.

You can also run Debezium on Kubernetes and OpenShift.

SQL Server connector configuration example

Following is an example of the configuration for a connector instance that captures data from a SQL Server server at port 1433 on 192.168.99.100, which we logically name fullfillment. Typically, you configure the Debezium SQL Server connector in a JSON file by setting the configuration properties that are available for the connector.

You can choose to produce events for a subset of the schemas and tables in a database. Optionally, you can ignore, mask, or truncate columns that contain sensitive data, that are larger than a specified size, or that you do not need.

  1. {
  2. "name": "inventory-connector", (1)
  3. "config": {
  4. "connector.class": "io.debezium.connector.sqlserver.SqlServerConnector", (2)
  5. "database.hostname": "192.168.99.100", (3)
  6. "database.port": "1433", (4)
  7. "database.user": "sa", (5)
  8. "database.password": "Password!", (6)
  9. "database.names": "testDB1,testDB2", (7)
  10. "topic.prefix": "fullfillment", (8)
  11. "table.include.list": "dbo.customers", (9)
  12. "schema.history.internal.kafka.bootstrap.servers": "kafka:9092", (10)
  13. "schema.history.internal.kafka.topic": "schemahistory.fullfillment", (11)
  14. "database.ssl.truststore": "path/to/trust-store", (12)
  15. "database.ssl.truststore.password": "password-for-trust-store" (13)
  16. }
  17. }
1The name of our connector when we register it with a Kafka Connect service.
2The name of this SQL Server connector class.
3The address of the SQL Server instance.
4The port number of the SQL Server instance.
5The name of the SQL Server user
6The password for the SQL Server user
7The name of the database to capture changes from.
8The topic prefix for the SQL Server instance/cluster, which forms a namespace and is used in all the names of the Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro converter is used.
9A list of all tables whose changes Debezium should capture.
10The list of Kafka brokers that this connector will use to write and recover DDL statements to the database schema history topic.
11The name of the database schema history topic where the connector will write and recover DDL statements. This topic is for internal use only and should not be used by consumers.
12The path to the SSL truststore that stores the server’s signer certificates. This property is required unless database encryption is disabled (database.encrypt=false).
13The SSL truststore password. This property is required unless database encryption is disabled (database.encrypt=false).

For the complete list of the configuration properties that you can set for the Debezium SQL Server connector, see SQL Server connector properties.

You can send this configuration with a POST command to a running Kafka Connect service. The service records the configuration and start up the one connector task that performs the following tasks:

  • Connects to the SQL Server database.

  • Reads the transaction log.

  • Records change events to Kafka topics.

Adding connector configuration

To start running a Debezium SQL Server connector, create a connector configuration, and add the configuration to your Kafka Connect cluster.

Prerequisites

Procedure

  1. Create a configuration for the SQL Server connector.

  2. Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.

Results

When the connector starts, it performs a consistent snapshot of the SQL Server databases that the connector is configured for. The connector then starts generating data change events for row-level operations and streaming the change event records to Kafka topics.

Connector properties

The Debezium SQL Server connector has numerous configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values.

Information about the properties is organized as follows:

Required Debezium SQL Server connector configuration properties

The following configuration properties are required unless a default value is available.

PropertyDefaultDescription

No default

Unique name for the connector. Attempting to register again with the same name will fail. (This property is required by all Kafka Connect connectors.)

No default

The name of the Java class for the connector. Always use a value of io.debezium.connector.sqlserver.SqlServerConnector for the SQL Server connector.

1

Specifies the maximum number of tasks that the connector can use to capture data from the database instance. If the database.names list contains more than one element, you can increase the value of this property to a number less than or equal to the number of elements in the list.

No default

IP address or hostname of the SQL Server database server.

1433

Integer port number of the SQL Server database server. If both database.port and database.instance are specified, database.instance is ignored. See JDBC driver for SQL server documentation for more details.

No default

Username to use when connecting to the SQL Server database server. Can be omitted when using Kerberos authentication, which can be configured using pass-through properties.

No default

Password to use when connecting to the SQL Server database server.

No default

Specifies the instance name of the SQL Server named instance. If both database.port and database.instance are specified, database.instance is ignored. See JDBC driver for SQL server documentation for more details.

No default

The comma-separated list of the SQL Server database names from which to stream the changes.

No default

Topic prefix that provides a namespace for the SQL Server database server that you want Debezium to capture. The prefix should be unique across all other connectors, since it is used as the prefix for all Kafka topic names that receive records from this connector. Only alphanumeric characters, hyphens, dots and underscores must be used in the database server logical name.

Do not change the value of this property. If you change the name value, after a restart, instead of continuing to emit events to the original topics, the connector emits subsequent events to topics whose names are based on the new value. The connector is also unable to recover its database schema history topic.

No default

An optional, comma-separated list of regular expressions that match names of schemas for which you want to capture changes. Any schema name not included in schema.include.list is excluded from having its changes captured. By default, the connector captures changes for all non-system schemas.

To match the name of a schema, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the schema; it does not match substrings that might be present in a schema name.
If you include this property in the configuration, do not also set the schema.exclude.list property.

No default

An optional, comma-separated list of regular expressions that match names of schemas for which you do not want to capture changes. Any schema whose name is not included in schema.exclude.list has its changes captured, with the exception of system schemas.

To match the name of a schema, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the schema; it does not match substrings that might be present in a schema name.
If you include this property in the configuration, do not set the schema.include.list property.

No default

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables that you want Debezium to capture. By default, the connector captures all non-system tables for the designated schemas. When this property is set, the connector captures changes only from the specified tables. Each identifier is of the form schemaName.tableName.

To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name.
If you include this property in the configuration, do not also set the table.exclude.list property.

No default

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for the tables that you want to exclude from being captured. Debezium captures all tables that are not included in table.exclude.list. Each identifier is of the form schemaName.tableName.

To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name.
If you include this property in the configuration, do not also set the table.include.list property.

empty string

An optional comma-separated list of regular expressions that match the fully-qualified names of columns that should be included in the change event message values. Fully-qualified names for columns are of the form schemaName.tableName.columnName.

Each change event record that Debezium emits for a table includes an event key that contains fields for each column in the table’s primary key or unique key. To ensure that event keys are generated correctly, if you set this property, be sure to explicitly list the primary key columns of any captured tables.

To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; it does not match substrings that might be present in a column name.
If you include this property in the configuration, do not also set the column.exclude.list property.

empty string

An optional comma-separated list of regular expressions that match the fully-qualified names of columns that should be excluded from change event message values. Fully-qualified names for columns are of the form schemaName.tableName.columnName. Note that primary key columns are always included in the event’s key, also if excluded from the value.

To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; it does not match substrings that might be present in a column name.
If you include this property in the configuration, do not also set the column.include.list property.

false

Specifies whether to skip publishing messages when there is no change in included columns. This would essentially filter messages if there is no change in columns included as per column.include.list or column.exclude.list properties.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form &lt;schemaName&gt;</em>.<em>&lt;tableName&gt;</em>.<em>&lt;columnName&gt;.
To match the name of a column Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name. In the resulting change event record, the values for the specified columns are replaced with pseudonyms.

A pseudonym consists of the hashed value that results from applying the specified hashAlgorithm and salt. Based on the hash function that is used, referential integrity is maintained, while column values are replaced with pseudonyms. Supported hash functions are described in the MessageDigest section of the Java Cryptography Architecture Standard Algorithm Name Documentation.

In the following example, CzQMA0cB5K is a randomly selected salt.

  1. column.mask.hash.SHA-256.with.salt.CzQMA0cB5K = inventory.orders.customerName, inventory.shipment.customerName

If necessary, the pseudonym is automatically shortened to the length of the column. The connector configuration can include multiple properties that specify different hash algorithms and salts.

Depending on the hashAlgorithm used, the salt selected, and the actual data set, the resulting data set might not be completely masked.

Hashing strategy version 2 should be used to ensure fidelity if the value is being hashed in different places or systems.

adaptive

Time, date, and timestamps can be represented with different kinds of precision, including: adaptive (the default) captures the time and timestamp values exactly as in the database using either millisecond, microsecond, or nanosecond precision values based on the database column’s type; or connect always represents time and timestamp values using Kafka Connect’s built-in representations for Time, Date, and Timestamp, which uses millisecond precision regardless of the database columns’ precision. For more information, see temporal values.

precise

Specifies how the connector should handle values for DECIMAL and NUMERIC columns:

precise (the default) represents them precisely using java.math.BigDecimal values represented in change events in a binary form.

double represents them using double values, which may result in a loss of precision but is easier to use.

string encodes values as formatted strings, which is easy to consume but semantic information about the real type is lost.

true

Boolean value that specifies whether the connector should publish changes in the database schema to a Kafka topic with the same name as the database server ID. Each schema change is recorded with a key that contains the database name and a value that is a JSON structure that describes the schema update. This is independent of how the connector internally records database schema history. The default is true.

true

Controls whether a delete event is followed by a tombstone event.

true - a delete operation is represented by a delete event and a subsequent tombstone event.

false - only a delete event is emitted.

After a source record is deleted, emitting a tombstone event (the default behavior) allows Kafka to completely delete all events that pertain to the key of the deleted row in case log compaction is enabled for the topic.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Set this property if you want to truncate the data in a set of columns when it exceeds the number of characters specified by the length in the property name. Set length to a positive integer value, for example, column.truncate.to.20.chars.

The fully-qualified name of a column observes the following format: <schemaName>.<tableName>.<columnName>. To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name.

You can specify multiple properties with different lengths in a single configuration.

n/a Fully-qualified names for columns are of the form schemaName.tableName.columnName.

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Set this property if you want the connector to mask the values for a set of columns, for example, if they contain sensitive data. Set length to a positive integer to replace data in the specified columns with the number of asterisk () characters specified by the length in the property name. Set length to 0 (zero) to replace data in the specified columns with an empty string.

The fully-qualified name of a column observes the following format: schemaName.tableName.columnName. To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name.

You can specify multiple properties with different lengths in a single configuration.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of columns for which you want the connector to emit extra parameters that represent column metadata. When this property is set, the connector adds the following fields to the schema of event records:

  • debezium.source.column.type

  • debezium.source.column.length

  • debezium.source.column.scale

These parameters propagate a column’s original type name and length (for variable-width types), respectively.
Enabling the connector to emit this extra data can assist in properly sizing specific numeric or character-based columns in sink databases.

The fully-qualified name of a column observes the following format: schemaName.tableName.columnName.
To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the column; the expression does not match substrings that might be present in a column name.

n/a

An optional, comma-separated list of regular expressions that specify the fully-qualified names of data types that are defined for columns in a database. When this property is set, for columns with matching data types, the connector emits event records that include the following extra fields in their schema:

  • debezium.source.column.type

  • debezium.source.column.length

  • debezium.source.column.scale

These parameters propagate a column’s original type name and length (for variable-width types), respectively.
Enabling the connector to emit this extra data can assist in properly sizing specific numeric or character-based columns in sink databases.

The fully-qualified name of a column observes the following format: schemaName.tableName.typeName.
To match the name of a data type, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the data type; the expression does not match substrings that might be present in a type name.

For the list of SQL Server-specific data type names, see the SQL Server data type mappings.

n/a

A list of expressions that specify the columns that the connector uses to form custom message keys for change event records that it publishes to the Kafka topics for specified tables.

By default, Debezium uses the primary key column of a table as the message key for records that it emits. In place of the default, or to specify a key for tables that lack a primary key, you can configure custom message keys based on one or more columns.

To establish a custom message key for a table, list the table, followed by the columns to use as the message key. Each list entry takes the following format:

<fully-qualified_tableName>:<keyColumn>,<keyColumn>

To base a table key on multiple column names, insert commas between the column names.

Each fully-qualified table name is a regular expression in the following format:

<schemaName>.<tableName>

The property can include entries for multiple tables. Use a semicolon to separate table entries in the list.

The following example sets the message key for the tables inventory.customers and purchase.orders:

inventory.customers:pk1,pk2;(.).purchaseorders:pk3,pk4

For the table inventory.customer, the columns pk1 and pk2 are specified as the message key. For the purchaseorders tables in any schema, the columns pk3 and pk4 server as the message key.

There is no limit to the number of columns that you use to create custom message keys. However, it’s best to use the minimum number that are required to specify a unique key.

bytes

Specifies how binary (binary, varbinary) columns should be represented in change events, including: bytes represents binary data as byte array (default), base64 represents binary data as base64-encoded String, base64-url-safe represents binary data as base64-url-safe-encoded String, hex represents binary data as hex-encoded (base16) String

none

Specifies how schema names should be adjusted for compatibility with the message converter used by the connector. Possible settings:

  • none does not apply any adjustment.

  • avro replaces the characters that cannot be used in the Avro type name with underscore.

  • avro_unicode replaces the underscore or characters that cannot be used in the Avro type name with corresponding unicode like _uxxxx. Note: is an escape sequence like backslash in Java

none

Specifies how field names should be adjusted for compatibility with the message converter used by the connector. Possible settings:

  • none does not apply any adjustment.

  • avro replaces the characters that cannot be used in the Avro type name with underscore.

  • avrounicode replaces the underscore or characters that cannot be used in the Avro type name with corresponding unicode like _uxxxx. Note: is an escape sequence like backslash in Java

For more information, see Avro naming.

Advanced SQL Server connector configuration properties

The following advanced configuration properties have good defaults that will work in most situations and therefore rarely need to be specified in the connector’s configuration.

PropertyDefaultDescription

No default

Enumerates a comma-separated list of the symbolic names of the custom converter instances that the connector can use. For example,

isbn

You must set the converters property to enable the connector to use a custom converter.

For each converter that you configure for a connector, you must also add a .type property, which specifies the fully-qualified name of the class that implements the converter interface. The .type property uses the following format:

<converterSymbolicName>.type

For example,

  1. isbn.type: io.debezium.test.IsbnConverter

If you want to further control the behavior of a configured converter, you can add one or more configuration parameters to pass values to the converter. To associate any additional configuration parameter with a converter, prefix the parameter names with the symbolic name of the converter. For example,

  1. isbn.schema.name: io.debezium.sqlserver.type.Isbn

initial

A mode for taking an initial snapshot of the structure and optionally data of captured tables. Once the snapshot is complete, the connector will continue reading change events from the database’s redo logs. The following values are supported:

    always

    Perform snapshot on each connector start. After the snapshot completes, the connector begins to stream event records for subsequent database changes.

    initial

    The connector performs a database snapshot as described in the default workflow for creating an initial snapshot. After the snapshot completes, the connector begins to stream event records for subsequent database changes.

    initialonly

    The connector performs a database snapshot and stops before streaming any change event records, not allowing any subsequent change events to be captured.

    schema_only

    Deprecated, see no_data.

    no_data

    The connector captures the structure of all relevant tables, performing all the steps described in the default snapshot workflow, except that it does not create READ events to represent the data set at the point of the connector’s start-up (Step 7.b).

    recovery

    Set this option to restore a database schema history topic that is lost or corrupted. After a restart, the connector runs a snapshot that rebuilds the topic from the source tables. You can also set the property to periodically prune a database schema history topic that experiences unexpected growth.

    Do not use this mode to perform a snapshot if schema changes were committed to the database after the last connector shutdown.
    when_needed

    After the connector starts, it performs a snapshot only if it detects one of the following circumstances:

    • It cannot detect any topic offsets.

    • A previously recorded offset specifies a log position that is not available on the server.

    configuration_based

    With this option, you control snapshot behavior through a set of connector properties that have the prefix ‘snapshot.mode.configuration.based’.

    custom

    The custom snapshot mode lets you inject your own implementation of the io.debezium.spi.snapshot.Snapshotter interface. Set the snapshot.mode.custom.name configuration property to the name provided by the name() method of your implementation.

For more information, see custom snapshotter SPI.

false

If the snapshot.mode is set to configuration_based, set this property to specify whether the connector includes table data when it performs a snapshot.

false

If the snapshot.mode is set to configuration_based, set this property to specify whether the connector includes the table schema when it performs a snapshot.

false

If the snapshot.mode is set to configuration_based, set this property to specify whether the connector begins to stream change events after a snapshot completes.

false

If the snapshot.mode is set to configuration_based, set this property to specify whether the connector includes table schema in a snapshot if the schema history topic is not available.

false

If the snapshot.mode is set to configuration_based, this property specifies whether the connector attempts to snapshot table data if it does not find the last committed offset in the transaction log.
Set the value to true to instruct the connector to perform a new snapshot.

No default

If snapshot.mode is set to custom, use this setting to specify the name of the custom implementation that is provided in the name() method that is defined in the ‘io.debezium.spi.snapshot.Snapshotter’ interface. After a connector restart, Debezium calls the specified custom implementation to determine whether to perform a snapshot. For more information, see custom snapshotter SPI.

exclusive

Controls whether and for how long the connector holds a table lock. Table locks prevent certain types of changes table operations from occurring while the connector performs a snapshot. You can set the following values:

    exclusive

    Controls how the connector holds locks on tables while performing the schema snapshot when snapshot.isolation.mode is REPEATABLE_READ or EXCLUSIVE.
    The connector will hold a table lock for exclusive table access for just the initial portion of the snapshot while the database schemas and other metadata are being read. The remaining work in a snapshot involves selecting all rows from each table, and this is done using a flashback query that requires no locks. However, in some cases it may be desirable to avoid locks entirely which can be done by specifying none. This mode is only safe to use if no schema changes are happening while the snapshot is taken.

    none

    Prevents the connector from acquiring any table locks during the snapshot. Use this setting only if no schema changes might occur during the creation of the snapshot.

    custom

    The connector performs a snapshot according to the implementation specified by the snapshot.locking.mode.custom.name property, which is a custom implementation of the io.debezium.spi.snapshot.SnapshotLock interface.

No default

When snapshot.locking.mode is set as custom, use this setting to specify the name of the custom implementation provided in the name() method that is defined by the ‘io.debezium.spi.snapshot.SnapshotLock’ interface. For more information, see custom snapshotter SPI.

select_all

Specifies how the connector queries data while performing a snapshot.
Set one of the following options:

    select_all

    The connector performs a select all query by default, optionally adjusting the columns selected based on the column include and exclude list configurations.

    custom

    The connector performs a snapshot query according to the implementation specified by the snapshot.query.mode.custom.name property, which defines a custom implementation of the io.debezium.spi.snapshot.SnapshotQuery interface.

This setting enables you to manage snapshot content in a more flexible manner compared to using the snapshot.select.statement.overrides property.

No default

When snapshot.query.mode is set to custom, use this setting to specify the name of the custom implementation provided in the name() method that is defined by the ‘io.debezium.spi.snapshot.SnapshotQuery’ interface. For more information, see custom snapshotter SPI.

All tables specified in table.include.list

An optional, comma-separated list of regular expressions that match the fully-qualified names (<dbName>.<schemaName>.<tableName>) of the tables to include in a snapshot. The specified items must be named in the connector’s table.include.list property. This property takes effect only if the connector’s snapshot.mode property is set to a value other than never.
This property does not affect the behavior of incremental snapshots.

To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression. That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name.

repeatable_read

Mode to control which transaction isolation level is used and how long the connector locks tables that are designated for capture. The following values are supported:

  • read_uncommitted

  • read_committed

  • repeatable_read

  • snapshot

  • exclusive (exclusive mode uses repeatable read isolation level, however, it takes the exclusive lock on all tables to be read).

The snapshot, read_committed and read_uncommitted modes do not prevent other transactions from updating table rows during initial snapshot. The exclusive and repeatable_read modes do prevent concurrent updates.

Mode choice also affects data consistency. Only exclusive and snapshot modes guarantee full consistency, that is, initial snapshot and streaming logs constitute a linear history. In case of repeatable_read and read_committed modes, it might happen that, for instance, a record added appears twice - once in initial snapshot and once in streaming phase. Nonetheless, that consistency level should do for data mirroring. For read_uncommitted there are no data consistency guarantees at all (some data might be lost or corrupted).

fail

Specifies how the connector should react to exceptions during processing of events. fail will propagate the exception (indicating the offset of the problematic event), causing the connector to stop.
warn will cause the problematic event to be skipped and the offset of the problematic event to be logged.
skip will cause the problematic event to be skipped.

500

Positive integer value that specifies the number of milliseconds the connector should wait during each iteration for new change events to appear. Defaults to 500 milliseconds, or 0.5 second.

8192

Positive integer value that specifies the maximum number of records that the blocking queue can hold. When Debezium reads events streamed from the database, it places the events in the blocking queue before it writes them to Kafka. The blocking queue can provide backpressure for reading change events from the database in cases where the connector ingests messages faster than it can write them to Kafka, or when Kafka becomes unavailable. Events that are held in the queue are disregarded when the connector periodically records offsets. Always set the value of max.queue.size to be larger than the value of max.batch.size.

0

A long integer value that specifies the maximum volume of the blocking queue in bytes. By default, volume limits are not specified for the blocking queue. To specify the number of bytes that the queue can consume, set this property to a positive long value.
If max.queue.size is also set, writing to the queue is blocked when the size of the queue reaches the limit specified by either property. For example, if you set max.queue.size=1000, and max.queue.size.in.bytes=5000, writing to the queue is blocked after the queue contains 1000 records, or after the volume of the records in the queue reaches 5000 bytes.

2048

Positive integer value that specifies the maximum size of each batch of events that should be processed during each iteration of this connector.

0

Controls how frequently heartbeat messages are sent.
This property contains an interval in milliseconds that defines how frequently the connector sends messages to a heartbeat topic. The property can be used to confirm whether the connector is still receiving change events from the database. You also should leverage heartbeat messages in cases where only records in non-captured tables are changed for a longer period of time. In such situation the connector would proceed to read the log from the database but never emit any change messages into Kafka, which in turn means that no offset updates are committed to Kafka. This may result in more change events to be re-sent after a connector restart. Set this parameter to 0 to not send heartbeat messages at all.
Disabled by default.

No default

An interval in milli-seconds that the connector should wait before taking a snapshot after starting up;
Can be used to avoid snapshot interruptions when starting multiple connectors in a cluster, which may cause re-balancing of connectors.

2000

Specifies the maximum number of rows that should be read in one go from each table while taking a snapshot. The connector will read the table contents in multiple batches of this size. Defaults to 2000.

No default

Specifies the number of rows that will be fetched for each database round-trip of a given query. Defaults to the JDBC driver’s default fetch size.

10000

An integer value that specifies the maximum amount of time (in milliseconds) to wait to obtain table locks when performing a snapshot. If table locks cannot be acquired in this time interval, the snapshot will fail (also see snapshots).
When set to 0 the connector will fail immediately when it cannot obtain the lock. Value -1 indicates infinite waiting.

No default

Specifies the table rows to include in a snapshot. Use the property if you want a snapshot to include only a subset of the rows in a table. This property affects snapshots only. It does not apply to events that the connector reads from the log.

The property contains a comma-separated list of fully-qualified table names in the form <schemaName>.<tableName>. For example,

“snapshot.select.statement.overrides”: “inventory.products,customers.orders”

For each table in the list, add a further configuration property that specifies the SELECT statement for the connector to run on the table when it takes a snapshot. The specified SELECT statement determines the subset of table rows to include in the snapshot. Use the following format to specify the name of this SELECT statement property:

snapshot.select.statement.overrides.<schemaName>.<tableName>. For example, snapshot.select.statement.overrides.customers.orders.

Example:

From a customers.orders table that includes the soft-delete column, delete_flag, add the following properties if you want a snapshot to include only those records that are not soft-deleted:

  1. snapshot.select.statement.overrides”: customer.orders”,
  2. snapshot.select.statement.overrides.customer.orders”: SELECT * FROM [customers].[orders] WHERE delete_flag = 0 ORDER BY id DESC

In the resulting snapshot, the connector includes only the records for which delete_flag = 0.

v2

Schema version for the source block in CDC events; Debezium 0.10 introduced a few breaking
changes to the structure of the source block in order to unify the exposed structure across all the connectors.
By setting this option to v1 the structure used in earlier versions can be produced. Note that this setting is not recommended and is planned for removal in a future Debezium version.

false

When set to true Debezium generates events with transaction boundaries and enriches data events envelope with transaction metadata.

10000 (10 seconds)

The number of milli-seconds to wait before restarting a connector after a retriable error occurs.

t

A comma-separated list of operation types that will be skipped during streaming. The operations include: c for inserts/create, u for updates, d for deletes, t for truncates, and none to not skip any operations. By default, truncate operations are skipped (not emitted by this connector).

No default value

Fully-qualified name of the data collection that is used to send signals to the connector.
Use the following format to specify the collection name:
<databaseName>.<schemaName>.<tableName>

source

List of the signaling channel names that are enabled for the connector. By default, the following channels are available:

No default

List of notification channel names that are enabled for the connector. By default, the following channels are available:

false

Allow schema changes during an incremental snapshot. When enabled the connector will detect schema change during an incremental snapshot and re-select a current chunk to avoid locking DDLs.

Note that changes to a primary key are not supported and can cause incorrect results if performed during an incremental snapshot. Another limitation is that if a schema change affects only columns’ default values, then the change won’t be detected until the DDL is processed from the transaction log stream. This doesn’t affect the snapshot events’ values, but the schema of snapshot events may have outdated defaults.

1024

The maximum number of rows that the connector fetches and reads into memory during an incremental snapshot chunk. Increasing the chunk size provides greater efficiency, because the snapshot runs fewer snapshot queries of a greater size. However, larger chunk sizes also require more memory to buffer the snapshot data. Adjust the chunk size to a value that provides the best performance in your environment.

insert_insert

Specifies the watermarking mechanism that the connector uses during an incremental snapshot to deduplicate events that might be captured by an incremental snapshot and then recaptured after streaming resumes.
You can specify one of the following options:

    insert_insert

    When you send a signal to initiate an incremental snapshot, for every chunk that Debezium reads during the snapshot, it writes an entry to the signaling data collection to record the signal to open the snapshot window. After the snapshot completes, Debezium inserts a second entry that records the signal to close the window.

    insert_delete

    When you send a signal to initiate an incremental snapshot, for every chunk that Debezium reads, it writes a single entry to the signaling data collection to record the signal to open the snapshot window. After the snapshot completes, this entry is removed. No entry is created for the signal to close the snapshot window. Set this option to prevent rapid growth of the signaling data collection.

0

Specifies the maximum number of transactions per iteration to be used to reduce the memory footprint when streaming changes from multiple tables in a database. When set to 0 (the default), the connector uses the current maximum LSN as the range to fetch changes from. When set to a value greater than zero, the connector uses the n-th LSN specified by this setting as the range to fetch changes from.

false

Uses OPTION(RECOMPILE) query option to all SELECT statements used during an incremental snapshot. This can help to solve parameter sniffing issues that may occur but can cause increased CPU load on the source database, depending on the frequency of query execution.

io.debezium.schema.SchemaTopicNamingStrategy

The name of the TopicNamingStrategy class that should be used to determine the topic name for data change, schema change, transaction, heartbeat event etc., defaults to SchemaTopicNamingStrategy.

.

Specify the delimiter for topic name, defaults to ..

10000

The size used for holding the topic names in bounded concurrent hash map. This cache will help to determine the topic name corresponding to a given data collection.

debezium-heartbeat

Controls the name of the topic to which the connector sends heartbeat messages. The topic name has this pattern:

topic.heartbeat.prefix.topic.prefix

For example, if the topic prefix is fulfillment, the default topic name is debezium-heartbeat.fulfillment.

transaction

Controls the name of the topic to which the connector sends transaction metadata messages. The topic name has this pattern:

topic.prefix.topic.transaction

For example, if the topic prefix is fulfillment, the default topic name is fulfillment.transaction.

For more information, see Transaction Metadata.

1

Specifies the number of threads that the connector uses when performing an initial snapshot. To enable parallel initial snapshots, set the property to a value greater than 1. In a parallel initial snapshot, the connector processes multiple tables concurrently. This feature is incubating.

No default

The custom metric tags will accept key-value pairs to customize the MBean object name which should be appended the end of regular name, each key would represent a tag for the MBean object name, and the corresponding value would be the value of that tag the key is. For example: k1=v1,k2=v2.

-1

Specifies how the connector responds after an operation that results in a retriable error, such as a connection error.
Set one of the following options:

    -1

    No limit. The connector always restarts automatically, and retries the operation, regardless of the number of previous failures.

    0

    Disabled. The connector fails immediately, and never retries the operation. User intervention is required to restart the connector.

    > 0

    The connector restarts automatically until it reaches the specified maximum number of retries. After the next failure, the connector stops, and user intervention is required to restart it.

function

Controls how the connector queries CDC data. The following modes are supported:

  • function: The data is queried by calling cdc.[fn_cdc_get_all_changes#] function. This is the default mode.

  • direct: Makes the connector to query change tables directly. Switching to direct mode and creating an index on ($start_lsn ASC, $seqval ASC, __$operation ASC) columns for each change table significantly speeds up querying CDC data.

Debezium SQL Server connector database schema history configuration properties

Debezium provides a set of schema.history.internal.* properties that control how the connector interacts with the schema history topic.

The following table describes the schema.history.internal properties for configuring the Debezium connector.

Table 17. Connector database schema history configuration properties
PropertyDefaultDescription

No default

The full name of the Kafka topic where the connector stores the database schema history.

No default

A list of host/port pairs that the connector uses for establishing an initial connection to the Kafka cluster. This connection is used for retrieving the database schema history previously stored by the connector, and for writing each DDL statement read from the source database. Each pair should point to the same Kafka cluster used by the Kafka Connect process.

100

An integer value that specifies the maximum number of milliseconds the connector should wait during startup/recovery while polling for persisted data. The default is 100ms.

3000

An integer value that specifies the maximum number of milliseconds the connector should wait while fetching cluster information using Kafka admin client.

30000

An integer value that specifies the maximum number of milliseconds the connector should wait while create kafka history topic using Kafka admin client.

100

The maximum number of times that the connector should try to read persisted history data before the connector recovery fails with an error. The maximum amount of time to wait after receiving no data is recovery.attempts × recovery.poll.interval.ms.

false

A Boolean value that specifies whether the connector should ignore malformed or unknown database statements or stop processing so a human can fix the issue. The safe default is false. Skipping should be used only with care as it can lead to data loss or mangling when the binlog is being processed.

false

A Boolean value that specifies whether the connector records schema structures from all tables in a schema or database, or only from tables that are designated for capture.
Specify one of the following values:

    false (default)

    During a database snapshot, the connector records the schema data for all non-system tables in the database, including tables that are not designated for capture. It’s best to retain the default setting. If you later decide to capture changes from tables that you did not originally designate for capture, the connector can easily begin to capture data from those tables, because their schema structure is already stored in the schema history topic. Debezium requires the schema history of a table so that it can identify the structure that was present at the time that a change event occurred.

    true

    During a database snapshot, the connector records the table schemas only for the tables from which Debezium captures change events. If you change the default value, and you later configure the connector to capture data from other tables in the database, the connector lacks the schema information that it requires to capture change events from the tables.

false

A Boolean value that specifies whether the connector records schema structures from all logical databases in the database instance.
Specify one of the following values:

    true

    The connector records schema structures only for tables in the logical database and schema from which Debezium captures change events.

    false

    The connector records schema structures for all logical databases.

The default value is true for MySQL Connector

Pass-through database schema history properties for configuring producer and consumer clients

Debezium relies on a Kafka producer to write schema changes to database schema history topics. Similarly, it relies on a Kafka consumer to read from database schema history topics when a connector starts. You define the configuration for the Kafka producer and consumer clients by assigning values to a set of pass-through configuration properties that begin with the schema.history.internal.producer.* and schema.history.internal.consumer.* prefixes. The pass-through producer and consumer database schema history properties control a range of behaviors, such as how these clients secure connections with the Kafka broker, as shown in the following example:

  1. schema.history.internal.producer.security.protocol=SSL
  2. schema.history.internal.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
  3. schema.history.internal.producer.ssl.keystore.password=test1234
  4. schema.history.internal.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
  5. schema.history.internal.producer.ssl.truststore.password=test1234
  6. schema.history.internal.producer.ssl.key.password=test1234
  7. schema.history.internal.consumer.security.protocol=SSL
  8. schema.history.internal.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
  9. schema.history.internal.consumer.ssl.keystore.password=test1234
  10. schema.history.internal.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
  11. schema.history.internal.consumer.ssl.truststore.password=test1234
  12. schema.history.internal.consumer.ssl.key.password=test1234

Debezium strips the prefix from the property name before it passes the property to the Kafka client.

See the Kafka documentation for more details about Kafka producer configuration properties and Kafka consumer configuration properties.

Debezium connector Kafka signals configuration properties

Debezium provides a set of signal.* properties that control how the connector interacts with the Kafka signals topic.

The following table describes the Kafka signal properties.

Table 18. Kafka signals configuration properties
PropertyDefaultDescription

<topic.prefix>-signal

The name of the Kafka topic that the connector monitors for ad hoc signals.

If automatic topic creation is disabled, you must manually create the required signaling topic. A signaling topic is required to preserve signal ordering. The signaling topic must have a single partition.

kafka-signal

The name of the group ID that is used by Kafka consumers.

No default

A list of host/port pairs that the connector uses for establishing an initial connection to the Kafka cluster. Each pair references the Kafka cluster that is used by the Debezium Kafka Connect process.

100

An integer value that specifies the maximum number of milliseconds that the connector waits when polling signals.

false

Enable the offset commit for the signal topic in order to guarantee At-Least-Once delivery. If disabled, only signals received when the consumer is up&running are processed. Any signals received when the consumer is down are lost.

Debezium connector pass-through signals Kafka consumer client configuration properties

The Debezium connector provides for pass-through configuration of the signals Kafka consumer. Pass-through signals properties begin with the prefix signals.consumer.*. For example, the connector passes properties such as signal.consumer.security.protocol=SSL to the Kafka consumer.

Debezium strips the prefixes from the properties before it passes the properties to the Kafka signals consumer.

Debezium connector sink notifications configuration properties

The following table describes the notification properties.

Table 19. Sink notification configuration properties
PropertyDefaultDescription

No default

The name of the topic that receives notifications from Debezium. This property is required when you configure the notification.enabled.channels property to include sink as one of the enabled notification channels.

Debezium SQL Server connector pass-through database driver configuration properties

The Debezium connector provides for pass-through configuration of the database driver. Pass-through database properties begin with the prefix driver.*. For example, the connector passes properties such as driver.foobar=false to the JDBC URL.

As is the case with the pass-through properties for database schema history clients, Debezium strips the prefixes from the properties before it passes them to the database driver.

Database schema evolution

When change data capture is enabled for a SQL Server table, as changes occur in the table, event records are persisted to a capture table on the server. If you introduce a change in the structure of the source table change, for example, by adding a new column, that change is not dynamically reflected in the change table. For as long as the capture table continues to use the outdated schema, the Debezium connector is unable to emit data change events for the table correctly. You must intervene to refresh the capture table to enable the connector to resume processing change events.

Because of the way that CDC is implemented in SQL Server, you cannot use Debezium to update capture tables. To refresh capture tables, one must be a SQL Server database operator with elevated privileges. As a Debezium user, you must coordinate tasks with the SQL Server database operator to complete the schema refresh and restore streaming to Kafka topics.

You can use one of the following methods to update capture tables after a schema change:

There are advantages and disadvantages to using each type of procedure.

Whether you use the online or offline update method, you must complete the entire schema update process before you apply subsequent schema updates on the same source table. The best practice is to execute all DDLs in a single batch so the procedure can be run only once.

Some schema changes are not supported on source tables that have CDC enabled. For example, if CDC is enabled on a table, SQL Server does not allow you to change the schema of the table if you renamed one of its columns or changed the column type.

After you change a column in a source table from NULL to NOT NULL or vice versa, the SQL Server connector cannot correctly capture the changed information until after you create a new capture instance. If you do not create a new capture table after a change to the column designation, change event records that the connector emits do not correctly indicate whether the column is optional. That is, columns that were previously defined as optional (or NULL) continue to be, despite now being defined as NOT NULL. Similarly, columns that had been defined as required (NOT NULL), retain that designation, although they are now defined as NULL.

After you rename a table using sp_rename function, it will continue to emit changes under the old source table name until the connector is restarted. Upon restart of the connector, it will emit changes under the new source table name.

Offline schema updates

Offline schema updates provide the safest method for updating capture tables. However, offline updates might not be feasible for use with applications that require high-availability.

Prerequisites

  • An update was committed to the schema of a SQL Server table that has CDC enabled.

  • You are a SQL Server database operator with elevated privileges.

Procedure

  1. Suspend the application that updates the database.

  2. Wait for the Debezium connector to stream all unstreamed change event records.

  3. Stop the Debezium connector.

  4. Apply all changes to the source table schema.

  5. Create a new capture table for the update source table using sys.sp_cdc_enable_table procedure with a unique value for parameter @capture_instance.

  6. Resume the application that you suspended in Step 1.

  7. Start the Debezium connector.

  8. After the Debezium connector starts streaming from the new capture table, drop the old capture table by running the stored procedure sys.sp_cdc_disable_table with the parameter @capture_instance set to the old capture instance name.

Online schema updates

The procedure for completing an online schema updates is simpler than the procedure for running an offline schema update, and you can complete it without requiring any downtime in application and data processing. However, with online schema updates, a potential processing gap can occur after you update the schema in the source database, but before you create the new capture instance. During that interval, change events continue to be captured by the old instance of the change table, and the change data that is saved to the old table retains the structure of the earlier schema. So, for example, if you added a new column to a source table, change events that are produced before the new capture table is ready, do not contain a field for the new column. If your application does not tolerate such a transition period, it is best to use the offline schema update procedure.

Prerequisites

  • An update was committed to the schema of a SQL Server table that has CDC enabled.

  • You are a SQL Server database operator with elevated privileges.

Procedure

  1. Apply all changes to the source table schema.

  2. Create a new capture table for the update source table by running the sys.sp_cdc_enable_table stored procedure with a unique value for the parameter @capture_instance.

  3. When Debezium starts streaming from the new capture table, you can drop the old capture table by running the sys.sp_cdc_disable_table stored procedure with the parameter @capture_instance set to the old capture instance name.

Example: Running an online schema update after a database schema change

Let’s deploy the SQL Server based Debezium tutorial to demonstrate the online schema update.

In the following example, a column phone_number is added to the customers table.

  1. Type the following command to start the database shell:
  1. docker-compose -f docker-compose-sqlserver.yaml exec sqlserver bash -c '/opt/mssql-tools/bin/sqlcmd -U sa -P $SA_PASSWORD -d testDB'
  1. Modify the schema of the customers source table by running the following query to add the phone_number field:

    1. ALTER TABLE customers ADD phone_number VARCHAR(32);
  2. Create the new capture instance by running the sys.sp_cdc_enable_table stored procedure.

    1. EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', @role_name = NULL, @supports_net_changes = 0, @capture_instance = 'dbo_customers_v2';
    2. GO
  3. Insert new data into the customers table by running the following query:

    1. INSERT INTO customers(first_name,last_name,email,phone_number) VALUES ('John','Doe','john.doe@example.com', '+1-555-123456');
    2. GO

    The Kafka Connect log reports on configuration updates through entries similar to the following message:

    1. connect_1 | 2019-01-17 10:11:14,924 INFO || Multiple capture instances present for the same table: Capture instance "dbo_customers" [sourceTableId=testDB.dbo.customers, changeTableId=testDB.cdc.dbo_customers_CT, startLsn=00000024:00000d98:0036, changeTableObjectId=1525580473, stopLsn=00000025:00000ef8:0048] and Capture instance "dbo_customers_v2" [sourceTableId=testDB.dbo.customers, changeTableId=testDB.cdc.dbo_customers_v2_CT, startLsn=00000025:00000ef8:0048, changeTableObjectId=1749581271, stopLsn=NULL] [io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource]
    2. connect_1 | 2019-01-17 10:11:14,924 INFO || Schema will be changed for ChangeTable [captureInstance=dbo_customers_v2, sourceTableId=testDB.dbo.customers, changeTableId=testDB.cdc.dbo_customers_v2_CT, startLsn=00000025:00000ef8:0048, changeTableObjectId=1749581271, stopLsn=NULL] [io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource]
    3. ...
    4. connect_1 | 2019-01-17 10:11:33,719 INFO || Migrating schema to ChangeTable [captureInstance=dbo_customers_v2, sourceTableId=testDB.dbo.customers, changeTableId=testDB.cdc.dbo_customers_v2_CT, startLsn=00000025:00000ef8:0048, changeTableObjectId=1749581271, stopLsn=NULL] [io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource]

    Eventually, the phone_number field is added to the schema and its value appears in messages written to the Kafka topic.

    1. ...
    2. {
    3. "type": "string",
    4. "optional": true,
    5. "field": "phone_number"
    6. }
    7. ...
    8. "after": {
    9. "id": 1005,
    10. "first_name": "John",
    11. "last_name": "Doe",
    12. "email": "john.doe@example.com",
    13. "phone_number": "+1-555-123456"
    14. },
  4. Drop the old capture instance by running the sys.sp_cdc_disable_table stored procedure.

    1. EXEC sys.sp_cdc_disable_table @source_schema = 'dbo', @source_name = 'dbo_customers', @capture_instance = 'dbo_customers';
    2. GO

Monitoring

The Debezium SQL Server connector provides three types of metrics that are in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect provide. The connector provides the following metrics:

For information about how to expose the preceding metrics through JMX, see the Debezium monitoring documentation.

Snapshot metrics

The MBean is debezium.sql_server:type=connector-metrics,server=_<topic.prefix>_,task=_<task.id>_,context=snapshot.

Snapshot metrics are not exposed unless a snapshot operation is active, or if a snapshot has occurred since the last connector start.

The following table lists the shapshot metrics that are available.

AttributesTypeDescription

string

The last snapshot event that the connector has read.

long

The number of milliseconds since the connector has read and processed the most recent event.

long

The total number of events that this connector has seen since last started or reset.

long

The number of events that have been filtered by include/exclude list filtering rules configured on the connector.

string[]

The list of tables that are captured by the connector.

int

The length the queue used to pass events between the snapshotter and the main Kafka Connect loop.

int

The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop.

int

The total number of tables that are being included in the snapshot.

int

The number of tables that the snapshot has yet to copy.

boolean

Whether the snapshot was started.

boolean

Whether the snapshot was paused.

boolean

Whether the snapshot was aborted.

boolean

Whether the snapshot completed.

long

The total number of seconds that the snapshot has taken so far, even if not complete. Includes also time when snapshot was paused.

long

The total number of seconds that the snapshot was paused. If the snapshot was paused several times, the paused time adds up.

Map<String, Long>

Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table.

long

The maximum buffer of the queue in bytes. This metric is available if max.queue.size.in.bytes is set to a positive long value.

long

The current volume, in bytes, of records in the queue.

The connector also provides the following additional snapshot metrics when an incremental snapshot is executed:

AttributesTypeDescription

string

The identifier of the current snapshot chunk.

string

The lower bound of the primary key set defining the current chunk.

string

The upper bound of the primary key set defining the current chunk.

string

The lower bound of the primary key set of the currently snapshotted table.

string

The upper bound of the primary key set of the currently snapshotted table.

Streaming metrics

The MBean is debezium.sql_server:type=connector-metrics,server=_<topic.prefix>_,task=_<task.id>_,context=streaming.

The following table lists the streaming metrics that are available.

AttributesTypeDescription

string

The last streaming event that the connector has read.

long

The number of milliseconds since the connector has read and processed the most recent event.

long

The total number of events that this connector has seen since the last start or metrics reset.

long

The total number of create events that this connector has seen since the last start or metrics reset.

long

The total number of update events that this connector has seen since the last start or metrics reset.

long

The total number of delete events that this connector has seen since the last start or metrics reset.

long

The number of events that have been filtered by include/exclude list filtering rules configured on the connector.

string[]

The list of tables that are captured by the connector.

int

The length the queue used to pass events between the streamer and the main Kafka Connect loop.

int

The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop.

boolean

Flag that denotes whether the connector is currently connected to the database server.

long

The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incoporate any differences between the clocks on the machines where the database server and the connector are running.

long

The number of processed transactions that were committed.

Map<String, String>

The coordinates of the last received event.

string

Transaction identifier of the last processed transaction.

long

The maximum buffer of the queue in bytes. This metric is available if max.queue.size.in.bytes is set to a positive long value.

long

The current volume, in bytes, of records in the queue.

Schema history metrics

The MBean is debezium.sql_server:type=connector-metrics,context=schema-history,server=_<topic.prefix>_.

The following table lists the schema history metrics that are available.

AttributesTypeDescription

string

One of STOPPED, RECOVERING (recovering history from the storage), RUNNING describing the state of the database schema history.

long

The time in epoch seconds at what recovery has started.

long

The number of changes that were read during recovery phase.

long

the total number of schema changes applied during recovery and runtime.

long

The number of milliseconds that elapsed since the last change was recovered from the history store.

long

The number of milliseconds that elapsed since the last change was applied.

string

The string representation of the last change recovered from the history store.

string

The string representation of the last applied change.