You are viewing documentation for an outdated version of Debezium.
If you want to view the latest stable version of this page, please go here.

Debezium connector for SQL Server

Want to help us further hone and improve it? Learn how.

Debezium’s SQL Server Connector can monitor and record the row-level changes in the schemas of a SQL Server database.

The first time it connects to a SQL Server database/cluster, it reads a consistent snapshot of all of the schemas. When that snapshot is complete, the connector continuously streams the changes that were committed to SQL Server and generates corresponding insert, update and delete events. All of the events for each table are recorded in a separate Kafka topic, where they can be easily consumed by applications and services.

Overview

The functionality of the connector is based upon change data capture feature provided by SQL Server Standard (since SQL Server 2016 SP1) or Enterprise edition. Using this mechanism a SQL Server capture process monitors all databases and tables the user is interested in and stores the changes into specifically created CDC tables that have stored procedure facade.

The database operator must enable CDC for the table(s) that should be captured by the connector. The connector then produces a change event for every row-level insert, update, and delete operation that was published via the CDC API, recording all the change events for each table in a separate Kafka topic. The client applications read the Kafka topics that correspond to the database tables they’re interested in following, and react to every row-level event it sees in those topics.

The database operator normally enables CDC in the mid-life of a database an/or table. This means that the connector does not have the complete history of all changes that have been made to the database. Therefore, when the SQL Server connector first connects to a particular SQL Server database, it starts by performing a consistent snapshot of each of the database schemas. After the connector completes the snapshot, it continues streaming changes from the exact point at which the snapshot was made. This way, we start with a consistent view of all of the data, yet continue reading without having lost any of the changes made while the snapshot was taking place.

The connector is also tolerant of failures. As the connector reads changes and produces events, it records the position in the database log (LSN / Log Sequence Number), that is associated with CDC record, with each event. If the connector stops for any reason (including communication failures, network problems, or crashes), upon restart it simply continues reading the CDC tables where it last left off. This includes snapshots: if the snapshot was not completed when the connector is stopped, upon restart it begins a new snapshot.

Setting up SQL Server

Before using the SQL Server connector to monitor the changes committed on SQL Server, first enable CDC on a monitored database. Please bear in mind that CDC cannot be enabled for the primary database.

-- ====
-- Enable Database for CDC template
-- ====
USE MyDB
GO
EXEC sys.sp_cdc_enable_db
GO

Then enable CDC for each table that you plan to monitor.

-- ====
-- Enable a Table Specifying Filegroup Option Template
-- ====
USE MyDB
GO

EXEC sys.sp_cdc_enable_table
@source_schema = N'dbo',
@source_name   = N'MyTable',
@role_name     = N'MyRole',
@filegroup_name = N'MyDB_CT',
@supports_net_changes = 0
GO

Verify that the user have access to the CDC table.

-- ====
-- Verify the user of the connector have access, this query should not have empty result
-- ====

EXEC sys.sp_cdc_help_change_data_capture
GO

If the result is empty then please make sure that the user has privileges to access both the capture instance and CDC tables.

SQL Server on Azure

The SQL Server plug-in has not been tested with SQL Server on Azure. We welcome any feedback from a user to try the plug-in with database in managed environments.

SQL Server Always On

The SQL Server plug-in can capture changes from an Always On read-only replica. A few pre-requisities are necessary to be fulfilled:

  • 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 must be set to ReadOnly. This is required by SQL Server. When Debezium detects this configuration option then it will:

    • set snapshot.isolation.mode to snapshot as this is the only one transaction isolation mode supported by read-only replicas

    • commit the (read-only) transaction in every execution of the streaming query loop, as this is necessary to get the latest view on CDC data

How the SQL Server connector works

Snapshots

SQL Server CDC is not designed to store the complete history of database changes. It is thus necessary that Debezium establishes the baseline of current database content and streams it to the Kafka. This is achieved via a process called snapshotting.

By default (snapshotting mode initial) the connector will upon the first startup perform an initial consistent snapshot of the database (meaning the structure and data within any tables to be captured as per the connector’s filter configuration).

Each snapshot consists of the following steps:

  1. Determine the tables to be captured

  2. Obtain a lock on each of the monitored tables to ensure that no structural changes can occur to any of the tables. The level of the lock is determined by snapshot.isolation.mode configuration option.

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

  4. Capture the structure of all relevant tables.

  5. Optionally release the locks obtained in step 2, i.e. the locks are held usually only for a short period of time.

  6. Scan all of the relevant database tables and schemas as valid at the LSN position read in step 3, and generate a READ event for each row and write that event to the appropriate table-specific Kafka topic.

  7. Record the successful completion of the snapshot in the connector offsets.

Reading the change data tables

Upon first start-up, the connector takes a structural snapshot of the structure of the captured tables and persists this information in its internal database history topic. Then the connector identifies a change table for each of the source tables and executes the main loop

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

  2. Order the read changes incrementally according to commit LSN and change LSN. This ensures that the changes are replayed by Debezium in the same order as were made to the database.

  3. Pass commit and change LSNs as offsets to Kafka Connect.

  4. Store the maximum LSN and repeat the loop.

After a restart, the connector will resume from the offset (commit and change LSNs) where it left off before.

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

Topic names

The SQL Server connector writes events for all insert, update, and delete operations on a single table to a single Kafka topic. The name of the Kafka topics always takes the form serverName.schemaName.tableName, where serverName is the logical name of the connector as specified with the database.server.name configuration property, schemaName is the name of the schema where the operation occurred, and tableName is the name of the database table on which the operation occurred.

For example, consider a SQL Server installation with an inventory database that contains four tables: products, products_on_hand, customers, and orders in schema dbo. If the connector monitoring this database were given a logical server name of fulfillment, then the connector would produce events on these four Kafka topics:

  • fulfillment.dbo.products

  • fulfillment.dbo.products_on_hand

  • fulfillment.dbo.customers

  • fulfillment.dbo.orders

Schema change topic

For a table for which CDC is enabled, the Debezium SQL Server connector stores the history of schema changes to that table in a database history topic. This topic reflects an internal connector state and you should not use it. If your application needs to track schema changes, there is a public schema change topic. The name of the schema change topic is the same as the logical server name specified in the connector configuration.

The format of 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:

  • 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.

A message to the schema change topic contains a logical representation of the table schema, for example:

{
  "schema": {
  ...
  },
  "payload": {
    "source": {
      "version": "1.3.1.Final",
      "connector": "sqlserver",
      "name": "server1",
      "ts_ms": 1588252618953,
      "snapshot": "true",
      "db": "testDB",
      "schema": "dbo",
      "table": "customers",
      "change_lsn": null,
      "commit_lsn": "00000025:00000d98:00a2",
      "event_serial_no": null
    },
    "databaseName": "testDB", (1)
    "schemaName": "dbo",
    "ddl": null, (2)
    "tableChanges": [ (3)
      {
        "type": "CREATE", (4)
        "id": "\"testDB\".\"dbo\".\"customers\"", (5)
        "table": { (6)
          "defaultCharsetName": null,
          "primaryKeyColumnNames": [ (7)
            "id"
          ],
          "columns": [ (8)
            {
              "name": "id",
              "jdbcType": 4,
              "nativeType": null,
              "typeName": "int identity",
              "typeExpression": "int identity",
              "charsetName": null,
              "length": 10,
              "scale": 0,
              "position": 1,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            },
            {
              "name": "first_name",
              "jdbcType": 12,
              "nativeType": null,
              "typeName": "varchar",
              "typeExpression": "varchar",
              "charsetName": null,
              "length": 255,
              "scale": null,
              "position": 2,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            },
            {
              "name": "last_name",
              "jdbcType": 12,
              "nativeType": null,
              "typeName": "varchar",
              "typeExpression": "varchar",
              "charsetName": null,
              "length": 255,
              "scale": null,
              "position": 3,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            },
            {
              "name": "email",
              "jdbcType": 12,
              "nativeType": null,
              "typeName": "varchar",
              "typeExpression": "varchar",
              "charsetName": null,
              "length": 255,
              "scale": null,
              "position": 4,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            }
          ]
        }
      }
    ]
  }
}
Table 1. Descriptions of fields in messages emitted to the schema change topic
Item Field name Description

1

databaseName
schemaName

Identifies the database and the schema that contain the change.

2

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.

3

tableChanges

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

4

type

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

  • CREATE - table created

  • ALTER - table modified

  • DROP - table deleted

5

id

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

6

table

Represents table metadata after the applied change.

7

primaryKeyColumnNames

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

8

columns

Metadata for each column in the changed table.

In messages 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:

{
  "schema": {
    "type": "struct",
    "fields": [
      {
        "type": "string",
        "optional": false,
        "field": "databaseName"
      }
    ],
    "optional": false,
    "name": "io.debezium.connector.sqlserver.SchemaChangeKey"
  },
  "payload": {
    "databaseName": "testDB"
  }
}

Change data 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 converver and you configure it to produce all four basic change event parts, change events have this structure:

{
 "schema": { (1)
   ...
  },
 "payload": { (2)
   ...
 },
 "schema": { (3)
   ...
 },
 "payload": { (4)
   ...
 },
}
Table 2. Overview of change event basic content
Item Field name Description

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. 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
CREATE TABLE customers (
  id INTEGER IDENTITY(1001,1) NOT NULL PRIMARY KEY,
  first_name VARCHAR(255) NOT NULL,
  last_name VARCHAR(255) NOT NULL,
  email VARCHAR(255) NOT NULL UNIQUE
);
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:

{
    "schema": { (1)
        "type": "struct",
        "fields": [ (2)
            {
                "type": "int32",
                "optional": false,
                "field": "id"
            }
        ],
        "optional": false, (3)
        "name": "server1.dbo.customers.Key" (4)
    },
    "payload": { (5)
        "id": 1004
    }
}
Table 3. Description of change event key
Item Field name Description

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​.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.

Although the column.exclude.list and column.include.list connector configuration properties allow you to capture only a subset of table columns, all columns in a primary or unique key are always included in the event’s key.

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:

CREATE TABLE customers (
  id INTEGER IDENTITY(1001,1) NOT NULL PRIMARY KEY,
  first_name VARCHAR(255) NOT NULL,
  last_name VARCHAR(255) NOT NULL,
  email VARCHAR(255) NOT NULL UNIQUE
);

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:

{
  "schema": { (1)
    "type": "struct",
    "fields": [
      {
        "type": "struct",
        "fields": [
          {
            "type": "int32",
            "optional": false,
            "field": "id"
          },
          {
            "type": "string",
            "optional": false,
            "field": "first_name"
          },
          {
            "type": "string",
            "optional": false,
            "field": "last_name"
          },
          {
            "type": "string",
            "optional": false,
            "field": "email"
          }
        ],
        "optional": true,
        "name": "server1.dbo.customers.Value", (2)
        "field": "before"
      },
      {
        "type": "struct",
        "fields": [
          {
            "type": "int32",
            "optional": false,
            "field": "id"
          },
          {
            "type": "string",
            "optional": false,
            "field": "first_name"
          },
          {
            "type": "string",
            "optional": false,
            "field": "last_name"
          },
          {
            "type": "string",
            "optional": false,
            "field": "email"
          }
        ],
        "optional": true,
        "name": "server1.dbo.customers.Value",
        "field": "after"
      },
      {
        "type": "struct",
        "fields": [
          {
            "type": "string",
            "optional": false,
            "field": "version"
          },
          {
            "type": "string",
            "optional": false,
            "field": "connector"
          },
          {
            "type": "string",
            "optional": false,
            "field": "name"
          },
          {
            "type": "int64",
            "optional": false,
            "field": "ts_sec"
          },
          {
            "type": "boolean",
            "optional": true,
            "default": false,
            "field": "snapshot"
          },
          {
            "type": "string",
            "optional": false,
            "field": "db"
          },
          {
            "type": "string",
            "optional": false,
            "field": "schema"
          },
          {
            "type": "string",
            "optional": false,
            "field": "table"
          },
          {
            "type": "string",
            "optional": true,
            "field": "change_lsn"
          },
          {
            "type": "string",
            "optional": true,
            "field": "commit_lsn"
          },
          {
            "type": "int64",
            "optional": true,
            "field": "event_serial_no"
          }
        ],
        "optional": false,
        "name": "io.debezium.connector.sqlserver.Source", (3)
        "field": "source"
      },
      {
        "type": "string",
        "optional": false,
        "field": "op"
      },
      {
        "type": "int64",
        "optional": true,
        "field": "ts_ms"
      }
    ],
    "optional": false,
    "name": "server1.dbo.customers.Envelope" (4)
  },
  "payload": { (5)
    "before": null, (6)
    "after": { (7)
      "id": 1005,
      "first_name": "john",
      "last_name": "doe",
      "email": "john.doe@example.org"
    },
    "source": { (8)
      "version": "1.3.1.Final",
      "connector": "sqlserver",
      "name": "server1",
      "ts_sec": 1559729468470,
      "snapshot": false,
      "db": "testDB",
      "schema": "dbo",
      "table": "customers",
      "change_lsn": "00000027:00000758:0003",
      "commit_lsn": "00000027:00000758:0005",
      "event_serial_no": "1"
    },
    "op": "c", (9)
    "ts_ms": 1559729471739 (10)
  }
}
Table 4. Descriptions of create event value fields
Item Field name Description

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.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.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

  • 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

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.

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:

{
  "schema": { ... },
  "payload": {
    "before": { (1)
      "id": 1005,
      "first_name": "john",
      "last_name": "doe",
      "email": "john.doe@example.org"
    },
    "after": { (2)
      "id": 1005,
      "first_name": "john",
      "last_name": "doe",
      "email": "noreply@example.org"
    },
    "source": { (3)
      "version": "1.3.1.Final",
      "connector": "sqlserver",
      "name": "server1",
      "ts_sec": 1559729995937,
      "snapshot": false,
      "db": "testDB",
      "schema": "dbo",
      "table": "customers",
      "change_lsn": "00000027:00000ac0:0002",
      "commit_lsn": "00000027:00000ac0:0007",
      "event_serial_no": "2"
    },
    "op": "u", (4)
    "ts_ms": 1559729998706
  }
}
Table 5. Descriptions of update event value fields
Item Field name Description

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

  • 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 evemts. 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.

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:

{
  "schema": { ... },
  },
  "payload": {
    "before": { <>
      "id": 1005,
      "first_name": "john",
      "last_name": "doe",
      "email": "noreply@example.org"
    },
    "after": null, (2)
    "source": { (3)
      "version": "1.3.1.Final",
      "connector": "sqlserver",
      "name": "server1",
      "ts_sec": 1559730445243,
      "snapshot": false,
      "db": "testDB",
      "schema": "dbo",
      "table": "customers",
      "change_lsn": "00000027:00000db0:0005",
      "commit_lsn": "00000027:00000db0:0007",
      "event_serial_no": "1"
    },
    "op": "d", (4)
    "ts_ms": 1559730450205 (5)
  }
}
Table 6. Descriptions of delete event value fields
Item Field name Description

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

  • 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

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.

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 represents tranaction metadata boundaries and enrich data messages.

Transaction boundaries

Debezium generates events for every transaction BEGIN and END. Every event contains

  • status - BEGIN or END

  • id - string representation of unique transaction identifier

  • 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 that provides number of events emitted by changes originating from given data collection

Following is an example of what a message looks like:

{
  "status": "BEGIN",
  "id": "00000025:00000d08:0025",
  "event_count": null,
  "data_collections": null
}

{
  "status": "END",
  "id": "00000025:00000d08:0025",
  "event_count": 2,
  "data_collections": [
    {
      "data_collection": "testDB.dbo.tablea",
      "event_count": 1
    },
    {
      "data_collection": "testDB.dbo.tableb",
      "event_count": 1
    }
  ]
}

The transaction events are written to the topic named <database.server.name>.transaction.

Data events 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

Following is an example of what a message looks like:

{
  "before": null,
  "after": {
    "pk": "2",
    "aa": "1"
  },
  "source": {
...
  },
  "op": "c",
  "ts_ms": "1580390884335",
  "transaction": {
    "id": "00000025:00000d08:0025",
    "total_order": "1",
    "data_collection_order": "1"
  }
}

Database schema evolution

Debezium is able to capture schema changes over time. Due to the way CDC is implemented in SQL Server, it is necessary to work in co-operation with a database operator in order to ensure the connector continues to produce data change events when the schema is updated.

As was already mentioned before, Debezium uses SQL Server’s change data capture functionality. This means that SQL Server creates a capture table that contains all changes executed on the source table. Unfortunately, the capture table is static and needs to be updated when the source table structure changes. This update is not done by the connector itself but must be executed by an operator with elevated privileges.

There are generally two procedures how to execute the schema change:

  • cold - this is executed when Debezium is stopped

  • hot - executed while Debezium is running

Both approaches have their own advantages and disadvantages.

In both cases, it is critically important to execute the procedure completely before a new schema update on the same source table is made. It is thus recommended to execute all DDLs in a single batch so the procedure is done only once.

Not all schema changes are supported when CDC is enabled for a source table. One such exception identified is renaming a column or changing its type, SQL Server will not allow executing the operation.

Although not required by SQL Server’s CDC mechanism itself, a new capture instance must be created when altering a column from NULL to NOT NULL or vice versa. This is required so that the SQL Server connector can pick up that changed information. Otherwise, emitted change events will have the optional value for the corresponding field (true or false) set to match the original value.

Cold schema update

This is the safest procedure but might not be feasible for applications with high-availability requirements. The operator should follow this sequence of steps

  1. Suspend the application that generates the database records

  2. Wait for Debezium to stream all unstreamed changes

  3. Stop the 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

  7. Start the connector

  8. When Debezium starts streaming from the new capture table it is possible to drop the old one using sys.sp_cdc_disable_table stored procedure with parameter @capture_instance set to the old capture instance name

Hot schema update

The hot schema update does not require any downtime in application and data processing. The procedure itself is also much simpler than in case of cold schema update

  1. Apply all changes to the source table schema

  2. 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

  3. When Debezium starts streaming from the new capture table it is possible to drop the old one using sys.sp_cdc_disable_table stored procedure with parameter @capture_instance set to the old capture instance name

The hot schema update has one drawback. There is a period of time between the database schema update and creating the new capture instance. All changes that will arrive during this period are captured by the old instance with the old structure. For instance this means that in case of a newly added column any change event produced during this time will not yet contain a field for that new column. If your application does not tolerate such a transition period we recommend to follow the cold schema update.

Example

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

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

# Start the database shell
docker-compose -f docker-compose-sqlserver.yaml exec sqlserver bash -c '/opt/mssql-tools/bin/sqlcmd -U sa -P $SA_PASSWORD -d testDB'
-- Modify the source table schema
ALTER TABLE customers ADD phone_number VARCHAR(32);

-- Create the new capture instance
EXEC sys.sp_cdc_enable_table @source_schema = 'dbo', @source_name = 'customers', @role_name = NULL, @supports_net_changes = 0, @capture_instance = 'dbo_customers_v2';
GO

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

Kafka Connect log will contain messages like these:

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]
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]
...
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, there is a new field in the schema and value of the messages written to the Kafka topic.

...
     {
        "type": "string",
        "optional": true,
        "field": "phone_number"
     }
...
    "after": {
      "id": 1005,
      "first_name": "John",
      "last_name": "Doe",
      "email": "john.doe@example.com",
      "phone_number": "+1-555-123456"
    },
-- Drop the old capture instance
EXEC sys.sp_cdc_disable_table @source_schema = 'dbo', @source_name = 'dbo_customers', @capture_instance = 'dbo_customers';
GO

Data types

As described above, the SQL Server connector represents the changes to rows with events that are structured like the table in which the row exist. The event contains a field for each column value, and how that value is represented in the event depends on the SQL data type of the column. This section describes this mapping.

The following table describes how the connector maps each of the SQL Server data types to a literal type and semantic type within the events' fields. Here, the literal type describes how the value is literally represented using Kafka Connect schema types, namely INT8, INT16, INT32, INT64, FLOAT32, FLOAT64, BOOLEAN, STRING, BYTES, ARRAY, MAP, and STRUCT. The 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.

SQL Server data type Literal 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 type Literal 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 type Literal 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

SQL Server data type Literal type (schema type) Semantic type (schema name)

NUMERIC[(P[,S])]

BYTES

org.apache.kafka.connect.data.Decimal

DECIMAL[(P[,S])]

BYTES

org.apache.kafka.connect.data.Decimal

SMALLMONEY

BYTES

org.apache.kafka.connect.data.Decimal

MONEY

BYTES

org.apache.kafka.connect.data.Decimal

The scale schema parameter contains an integer that represents how many digits the decimal point was shifted. The connect.decimal.precision schema parameter contains an integer that represents the precision of the given decimal value.

Deployment

With Zookeeper, Kafka, and Kafka Connect installed, the remaining tasks to deploy a Debezium SQL Server connector are to download the connector’s plug-in archive, extract the JAR files into your Kafka Connect environment, and add the directory with the JAR files to Kafka Connect’s plugin.path. 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 Zookeeper, Kafka, SQL Server and Kafka Connect with the SQL Server connector already installed and ready to run. You can also run Debezium on Kubernetes and OpenShift.

Example configuration

To use the connector to produce change events for a particular SQL Server database or cluster:

  1. Enable the CDC on SQL Server to publish the CDC events in the database.

  2. Create a configuration file for the SQL Server connector.

When the connector starts, it will grab a consistent snapshot of the schemas in your SQL Server database and start streaming changes, producing events for every inserted, updated, and deleted row. You can also choose to produce events for a subset of the schemas and tables. Optionally ignore, mask, or truncate columns that are sensitive, too large, or not needed.

Following is an example of the configuration for a connector instance that monitors 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 using the configuration properties available for the connector.

{
  "name": "inventory-connector", (1)
  "config": {
    "connector.class": "io.debezium.connector.sqlserver.SqlServerConnector", (2)
    "database.hostname": "192.168.99.100", (3)
    "database.port": "1433", (4)
    "database.user": "sa", (5)
    "database.password": "Password!", (6)
    "database.dbname": "testDB", (7)
    "database.server.name": "fullfillment", (8)
    "table.include.list": "dbo.customers", (9)
    "database.history.kafka.bootstrap.servers": "kafka:9092", (10)
    "database.history.kafka.topic": "dbhistory.fullfillment" (11)
  }
}
1 The name of our connector when we register it with a Kafka Connect service.
2 The name of this SQL Server connector class.
3 The address of the SQL Server instance.
4 The port number of the SQL Server instance.
5 The name of the SQL Server user
6 The password for the SQL Server user
7 The name of the database to capture changes from.
8 The logical name of 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.
9 A list of all tables whose changes Debezium should capture.
10 The list of Kafka brokers that this connector will use to write and recover DDL statements to the database history topic.
11 The name of the database 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.

See the complete list of connector properties that can be specified in these configurations.

This configuration can be sent via POST to a running Kafka Connect service, which will then record the configuration and start up the one connector task that will connect to the SQL Server database, read the transaction log, and record events to Kafka topics.

Adding connector configuration

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

Prerequisites
  • SQL Server is set up to run a Debezium connector.

  • A Debezium SQL Server connector is installed.

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 change event records to Kafka topics.

Monitoring

The Debezium SQL Server connector has three metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.

Please refer to the monitoring documentation for details of how to expose these metrics via JMX.

Snapshot Metrics

The MBean is debezium.sql_server:type=connector-metrics,context=snapshot,server=<database.server.name>.

Attributes Type Description

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 monitored 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 aborted.

boolean

Whether the snapshot completed.

long

The total number of seconds that the snapshot has taken so far, even if not complete.

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.

Streaming Metrics

The MBean is debezium.sql_server:type=connector-metrics,context=streaming,server=<database.server.name>.

Attributes Type Description

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 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 monitored 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.

Schema History Metrics

The MBean is debezium.sql_server:type=connector-metrics,context=schema-history,server=<database.server.name>.

Attributes Type Description

string

One of STOPPED, RECOVERING (recovering history from the storage), RUNNING describing the state of the database 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.

Connector properties

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

Property Default Description

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

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

The maximum number of tasks that should be created for this connector. The SQL Server connector always uses a single task and therefore does not use this value, so the default is always acceptable.

IP address or hostname of the SQL Server database server.

1433

Integer port number of the SQL Server database server.

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

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

The name of the SQL Server database from which to stream the changes

Logical name that identifies and provides a namespace for the particular SQL Server database server being monitored. The logical name should be unique across all other connectors, since it is used as a prefix for all Kafka topic names emanating from this connector. Only alphanumeric characters and underscores should be used.

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

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

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be monitored; any table not included in table.include.list is excluded from monitoring. Each identifier is of the form schemaName.tableName. By default the connector will monitor every non-system table in each monitored schema. Must not be used with table.exclude.list.

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be excluded from monitoring; any table not included in table.exclude.list is monitored. Each identifier is of the form schemaName.tableName. Must not be used with table.include.list.

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. Note that primary key columns are always included in the event’s key, even if not included in the value. 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. Do not also set the column.include.list property.

n/a

An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be pseudonyms in the change event message values with a field value consisting of the hashed value using the algorithm hashAlgorithm and salt salt. Based on the used hash function referential integrity is kept while data is pseudonymized. Supported hash functions are described in the MessageDigest section of the Java Cryptography Architecture Standard Algorithm Name Documentation. The hash is automatically shortened to the length of the column.

Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer or zero. Fully-qualified names for columns are of the form schemaName.tableName.columnName.

Example:

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

where CzQMA0cB5K is a randomly selected salt.

Note: Depending on the hashAlgorithm used, the salt selected and the actual data set, the resulting masked data set may not be completely anonymized.

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. See temporal values.

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 history. The default is true.

true

Controls whether a tombstone event should be generated after a delete event.
When true the delete operations are represented by a delete event and a subsequent tombstone event. When false only a delete event is sent.
Emitting the tombstone event (the default behavior) allows Kafka to completely delete all events pertaining to the given key once the source record got deleted.

n/a

An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be truncated in the change event message values if the field values are longer than the specified number of characters. Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer. Fully-qualified names for columns are of the form schemaName.tableName.columnName.

n/a

An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be replaced in the change event message values with a field value consisting of the specified number of asterisk (*) characters. Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer or zero. Fully-qualified names for columns are of the form schemaName.tableName.columnName.

n/a

An optional comma-separated list of regular expressions that match the fully-qualified names of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change messages. The schema parameters __debezium.source.column.type, __debezium.source.column.length and __debezium.source.column.scale is used to propagate the original type name and length (for variable-width types), respectively. Useful to properly size corresponding columns in sink databases. Fully-qualified names for columns are of the form schemaName.tableName.columnName.

n/a

An optional comma-separated list of regular expressions that match the database-specific data type name of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change messages. The schema parameters __debezium.source.column.type, __debezium.source.column.length and __debezium.source.column.scale will be used to propagate the original type name and length (for variable-width types), respectively. Useful to properly size corresponding columns in sink databases. Fully-qualified data type names are of the form schemaName.tableName.typeName. See SQL Server data types for the list of SQL Server-specific data type names.

empty string

A semi-colon list of regular expressions that match fully-qualified tables and columns to map a primary key.
Each item (regular expression) must match the fully-qualified <fully-qualified table>:<a comma-separated list of columns> representing the custom key.
Fully-qualified tables could be defined as schemaName.tableName.

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.

Property Default Description

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.

Supported values are:
initial: Takes a snapshot of structure and data of captured tables; useful if topics should be populated with a complete representation of the data from the captured tables.
initial_only: Takes a snapshot of structure and data like initial but instead does not transition into streaming changes once the snapshot has completed.
schema_only: Takes a snapshot of the structure of captured tables only; useful if only changes happening from now onwards should be propagated to topics.

repeatable_read

Mode to control which transaction isolation level is used and how long the connector locks the monitored tables. There are five possible values: read_uncommitted, read_committed, repeatable_read, snapshot, and exclusive ( in fact, exclusive mode uses repeatable read isolation level, however, it takes the exclusive lock on all tables to be read).

It is worth documenting that snapshot, read_committed and read_uncommitted modes do not prevent other transactions from updating table rows during initial snapshot, while exclusive and repeatable_read do.

Another aspect is 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).

commit

String representing the criteria of the attached timestamp within the source record (ts_ms). commit will set the source timestamp to the instant where the record was committed in the database (default and current behavior). processing will set the source timestamp to the instant where the record was processed by Debezium. This option could be used when either we want to set the top level ts_ms value here or when we want to skip the query to extract the timestamp of that LSN.

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.

1000

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

8192

Positive integer value that specifies the maximum size of the blocking queue into which change events read from the database log are placed before they are written to Kafka. This queue can provide backpressure to the CDC table reader when, for example, writes to Kafka are slower or if Kafka is not available. Events that appear in the queue are not included in the offsets periodically recorded by this connector. Defaults to 8192, and should always be larger than the maximum batch size specified in the max.batch.size property.

2048

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

0

Controls how frequently heartbeat messages are sent.
This property contains an interval in milli-seconds that defines how frequently the connector sends messages into a heartbeat topic. This can be used to monitor 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.

__debezium-heartbeat

Controls the naming of the topic to which heartbeat messages are sent.
The topic is named according to the pattern <heartbeat.topics.prefix>.<server.name>.

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.

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.

Controls which rows from tables are included in snapshot.
This property contains a comma-separated list of fully-qualified tables (SCHEMA_NAME.TABLE_NAME). Select statements for the individual tables are specified in further configuration properties, one for each table, identified by the id snapshot.select.statement.overrides.[SCHEMA_NAME].[TABLE_NAME]. The value of those properties is the SELECT statement to use when retrieving data from the specific table during snapshotting. A possible use case for large append-only tables is setting a specific point where to start (resume) snapshotting, in case a previous snapshotting was interrupted.
Note: This setting has impact on snapshots only. Events captured during log reading are not affected by it.

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.

true when connector configuration explicitly specifies the key.converter or value.converter parameters to use Avro, otherwise defaults to false.

Whether field names are sanitized to adhere to Avro naming requirements. See Avro naming for more details.

Timezone of the server.

This is used to define the timezone of the transaction timestamp (ts_ms) retrieved from the server (which is actually not zoned). Default value is unset. Should only be specified when running on SQL Server 2014 or older and using different timezones for the database server and the JVM running the Debezium connector.
When unset, default behavior is to use the timezone of the VM running the Debezium connector. In this case, when running on on SQL Server 2014 or older and using different timezones on server and the connector, incorrect ts_ms values may be produced.
Possible values include "Z", "UTC", offset values like "+02:00", short zone ids like "CET", and long zone ids like "Europe/Paris".

false

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

See Transaction Metadata for additional details.

10000 (10 seconds)

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

The connector also supports pass-through configuration properties that are used when creating the Kafka producer and consumer. Specifically, all connector configuration properties that begin with the database.history.producer. prefix are used (without the prefix) when creating the Kafka producer that writes to the database history, and all those that begin with the prefix database.history.consumer. are used (without the prefix) when creating the Kafka consumer that reads the database history upon connector startup.

For example, the following connector configuration properties can be used to secure connections to the Kafka broker:

In addition to the pass-through to the Kafka producer and consumer, the properties starting with database., e.g. database.applicationName=debezium are passed to the JDBC URL.

database.history.producer.security.protocol=SSL
database.history.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
database.history.producer.ssl.keystore.password=test1234
database.history.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
database.history.producer.ssl.truststore.password=test1234
database.history.producer.ssl.key.password=test1234
database.history.consumer.security.protocol=SSL
database.history.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
database.history.consumer.ssl.keystore.password=test1234
database.history.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
database.history.consumer.ssl.truststore.password=test1234
database.history.consumer.ssl.key.password=test1234

Be sure to consult the Kafka documentation for all of the configuration properties for Kafka producers and consumers. (The SQL Server connector does use the new consumer.)