Technology Preview features are not supported with Red Hat production service-level agreements (SLAs) and might not be functionally complete; therefore, Red Hat does not recommend implementing any Technology Preview features in production environments. This Technology Preview feature provides early access to upcoming product innovations, enabling you to test functionality and provide feedback during the development process.
For more information about support scope, see link:https://access.redhat.com/support/offerings/techpreview/[Technology Preview Features Support Scope^].
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.
The functionality of the connector is based upon https://docs.microsoft.com/en-us/sql/relational-databases/track-changes/about-change-data-capture-sql-server?view=sql-server-2017[change data capture] feature provided by SQL Server Standard (https://blogs.msdn.microsoft.com/sqlreleaseservices/sql-server-2016-service-pack-1-sp1-released/[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 connector has been tested with SQL Server 2017, but community members have reportedly used it successfully with earlier versions up to 2014, too (as long as the CDC feature is provided).
The database operator must https://docs.microsoft.com/en-us/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server?view=sql-server-2017[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 won't 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 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.
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:
The format of the schema change topic messages is in incubating state and it can change without further notice.
====
SQL Server connector stores the historic schema structure of database tables in a database history topic.
This topic should be considered as an internal state of the connector and should not be used by the user.
If the application needs to track changes in the source database there is the public-facing schema change topic.
The topic name is the same as the logical server name configured in connector configuration.
Debezium emits a new message to this topic whenever a new table is streamed from or when the structure of the table is altered (link:#schema-evolution[schema evolution procedure must be followed]).
All data change events produced by the SQL Server connector have a key and a value, although the structure of the key and value depend on the table from which the change events originated (see link:#topic-names-sqlserver[Topic names]).
The SQL Server connector ensures that all Kafka Connect _schema names_ are http://avro.apache.org/docs/current/spec.html#names[valid Avro schema names].
This means that the logical server name must start with Latin letters or an underscore (e.g., [a-z,A-Z,\_]),
and the remaining characters in the logical server name and all characters in the schema and table names must be Latin letters, digits, or an underscore (e.g., [a-z,A-Z,0-9,\_]).
If not, then all invalid characters will automatically be replaced with an underscore character.
This can lead to unexpected conflicts when the logical server name, schema names, and table names contain other characters, and the only distinguishing characters between table full names are invalid and thus replaced with underscores.
This could be difficult for consumers to deal with, so to make it easy Kafka Connect makes each event self-contained.
Every message key and value has two parts: a _schema_ and _payload_.
The schema describes the structure of the payload, while the payload contains the actual data.
[[change-event-keys]]
==== Change Event Keys
For a given table, the change event's key will have a structure that contains a field for each column in the primary key (or unique key constraint) of the table at the time the event was created.
Consider a `customers` table defined in the `inventory` database's schema `dbo`:
[source,sql,indent=0]
----
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
);
----
If the `database.server.name` configuration property has the value `server1`,
every change event for the `customers` table while it has this definition will feature the same key structure, which in JSON looks like this:
[source,json,indent=0]
----
{
"schema": {
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "id"
}
],
"optional": false,
"name": "server1.dbo.customers.Key"
},
"payload": {
"id": 1004
}
}
----
The `schema` portion of the key contains a Kafka Connect schema describing what is in the key portion, and in our case that means that the `payload` value is not optional, is a structure defined by a schema named `server1.dbo.customers.Key`, and has one required field named `id` of type `int32`.
If we look at the value of the key's `payload` field, we'll see that it is indeed a structure (which in JSON is just an object) with a single `id` field, whose value is `1004`.
Therefore, we interpret this key as describing the row in the `dbo.customers` table (output from the connector named `server1`) whose `id` primary key column had a value of `1004`.
Although the `column.blacklist` configuration property allows you to remove columns from the event values, all columns in a primary or unique key are always included in the event's key.
====
[WARNING]
====
If the table does not have a primary or unique key, then the change event's key will be null. This makes sense since the rows in a table without a primary or unique key constraint cannot be uniquely identified.
Like the message key, the value of a change event message has a _schema_ section and _payload_ section.
The payload section of every change event value produced by the SQL Server connector has an _envelope_ structure with the following fields:
* `op` is a mandatory field that contains a string value describing the type of operation. Values for the SQL Server connector are `c` for create (or insert), `u` for update, `d` for delete, and `r` for read (in the case of a snapshot).
* `before` is an optional field that if present contains the state of the row _before_ the event occurred. The structure is described by the `server1.dbo.customers.Value` Kafka Connect schema, which the `server1` connector uses for all rows in the `dbo.customers` table.
* `after` is an optional field that if present contains the state of the row _after_ the event occurred. The structure is described by the same `server1.dbo.customers.Value` Kafka Connect schema used in `before`.
* `source` is a mandatory field that contains a structure describing the source metadata for the event, which in the case of SQL Server contains these fields: the {prodname} version, the connector name, whether the event is part of an ongoing snapshot or not, the commit LSN (not while snapshotting), the LSN of the change, database, schema and table where the change happened, and a timestamp representing the point in time when the record was changed in the source database (during snapshotting, it'll be the point in time of snapshotting).
There are mostly two situations when you can see it present with value different from `1`:
+
** update events will have the value set to `2`, this is because the update generates two events in the CDC change table of SQL Server (https://docs.microsoft.com/en-us/sql/relational-databases/system-tables/cdc-capture-instance-ct-transact-sql?view=sql-server-2017[source documentation]).
** when a primary key is updated, then SQL Server emits two records - `delete` to remove the record with the old primary key value and `insert` to create the record with the new primary key.
* `ts_ms` is optional and if present contains the time (using the system clock in the JVM running the Kafka Connect task) at which the connector processed the event.
And of course, the _schema_ portion of the event message's value contains a schema that describes this envelope structure and the nested fields within it.
If we look at the `schema` portion of this event's _value_, we can see the schema for the _envelope_, the schema for the `source` structure (which is specific to the SQL Server connector and reused across all events), and the table-specific schemas for the `before` and `after` fields.
The names of the schemas for the `before` and `after` fields are of the form _logicalName_._schemaName_._tableName_.Value, and thus are entirely independent from all other schemas for all other tables.
This means that when using the Avro Converter, the resulting Avro schemas for _each table_ in each _logical source_ have their own evolution and history.
If we look at the `payload` portion of this event's _value_, we can see the information in the event, namely that it is describing that the row was created (since `op=c`), and that the `after` field value contains the values of the new inserted row's' `id`, `first_name`, `last_name`, and `email` columns.
The value of an _update_ change event on this table will actually have the exact same _schema_, and its payload is structured the same but will hold different values.
When we compare this to the value in the _insert_ event, we see a couple of differences in the `payload` section:
* The `op` field value is now `u`, signifying that this row changed because of an update
* The `before` field now has the state of the row with the values before the database commit
* The `after` field now has the updated state of the row, and here was can see that the `email` value is now `noreply@example.org`.
* The `source` field structure has the same fields as before, but the values are different since this event is from a different position in the transaction log.
That is due to the update event composed of two events behind the scenes and we are exposing only the second one.
If you are interested in details please check the https://docs.microsoft.com/en-us/sql/relational-databases/system-tables/cdc-capture-instance-ct-transact-sql?view=sql-server-2017[source documentation] and refer to the field `$operation`.
There are several things we can learn by just looking at this `payload` section. We can compare the `before` and `after` structures to determine what actually changed in this row because of the commit.
The `source` structure tells us information about SQL Server's record of this change (providing traceability), but more importantly this has information we can compare to other events in this and other topics to know whether this event occurred before, after, or as part of the same SQL Server commit as other events.
When the columns for a row's primary/unique key are updated, the value of the row's key has changed so {prodname} will output _three_ events: a `DELETE` event and a link:#sqlserver-tombstone-events[tombstone event] with the old key for the row, followed by an `INSERT` event with the new key for the row.
Now, let's look at the value of a _delete_ event for the same table. Once again, the `schema` portion of the value is exactly the same as with the _create_ and _update_ events:
When a row is deleted, the _delete_ event value listed above still works with log compaction, since Kafka can still remove all earlier messages with that same key.
But only if the message value is `null` will Kafka know that it can remove _all messages_ with that same key.
To make this possible, the SQL Server connector always follows the _delete_ event with a special _tombstone_ event that has the same key but `null` value.
This feature is under active development right now (incubating),
so the structure of transaction events or other details may still change as development progresses.
====
Debezium can generate events that represents tranaction metadata boundaries and enrich data messages.
==== Transaction boundaries
Debezium generates events for every transaction start 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
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.
Both approaches have their own advantages and disadvantages.
[WARNING]
====
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.
====
[NOTE]
====
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.
====
[NOTE]
====
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.
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`
8. When {prodname} 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 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 {prodname} 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
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.
Let's deploy the SQL Server based https://github.com/debezium/debezium-examples/tree/master/tutorial#using-sql-server[Debezium tutorial] to demonstrate the hot schema update.
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]
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.
Passing the default value helps though with satisfying the compatibility rules when xref:configuration/avro.adoc[using Avro] as serialization format together with the Confluent schema registry.
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:
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:
| 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 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 epoch, and does not include timezone information.
|`SMALLDATETIME`
|`INT64`
|`org.apache.kafka.connect.data.Timestamp`
| Represents the number of milliseconds past epoch, and does not include timezone information.
|`DATETIME2`
|`INT64`
|`org.apache.kafka.connect.data.Timestamp`
| Represents the number of milliseconds since epoch, and does not include timezone information. SQL Server allows `P` to be in the range 0-7 to store up to tenth of microsecond precision, though this mode results in a loss of precision when `P` > 3.
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".
If you've already installed https://zookeeper.apache.org[Zookeeper], http://kafka.apache.org/[Kafka], and {link-kafka-docs}.html#connect[Kafka Connect], then using Debezium's SQL Server` connector is easy.
Simply download the https://repo1.maven.org/maven2/io/debezium/debezium-connector-sqlserver/{debezium-version}/debezium-connector-sqlserver-{debezium-version}-plugin.tar.gz[connector's plugin archive], extract the JARs into your Kafka Connect environment, and add the directory with the JARs to {link-kafka-docs}/#connectconfigs[Kafka Connect's plugin.path].
Installing the SQL Server connector is a simple process whereby you only need to download the JAR, extract it to your Kafka Connect environment, and ensure the plugin's parent directory is specified in your Kafka Connect environment.
* You have link:https://zookeeper.apache.org/[Zookeeper], link:http://kafka.apache.org/[Kafka], and link:{link-kafka-docs}.html#connect[Kafka Connect] installed.
. Download the {prodname} link:https://access.redhat.com/jbossnetwork/restricted/listSoftware.html?product=red.hat.integration&downloadType=distributions[SQL Server connector].
* link:https://access.redhat.com/documentation/en-us/red_hat_integration/2019-12/html-single/installing_change_data_capture_on_openshift[Installing {prodname} on OpenShift]
* link:https://access.redhat.com/documentation/en-us/red_hat_integration/2019-12/html-single/installing_change_data_capture_on_rhel[Installing {prodname} on RHEL]
If immutable containers are your thing, then check out https://hub.docker.com/r/debezium/[Debezium's Docker images] for Zookeeper, Kafka and Kafka Connect with the SQL Server connector already pre-installed and ready to go.
You can even link:/docs/openshift/[run Debezium on OpenShift].
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`.
<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 Connector is used.
<9> A list of all tables whose changes {prodname} 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.
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`.
<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 Connector is used.
<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.
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.
The {prodname} SQL Server connector has three metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.
* <<snapshot-metrics, snapshot metrics>>; for monitoring the connector when performing snapshots
* <<streaming-metrics, streaming metrics>>; for monitoring the connector when reading CDC table data
* <<schema-history-metrics, schema history metrics>>; for monitoring the status of the connector's schema history
Please refer to the xref:operations/monitoring.adoc[monitoring documentation] for details of how to expose these metrics via JMX.
|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.
|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.
|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.
|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 the whitelist 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. May not be used with `table.blacklist`.
|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 the blacklist is monitored.
Each identifier is of the form _schemaName_._tableName_. May not be used with `table.whitelist`.
|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 blacklisted from the value.
|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 {link-java7-standard-names}[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 _databaseName_._schemaName_._tableName_._columnName_.
Note: Depending on the `_hashAlgorithm_` used, the `_salt_` selected and the actual data set, the resulting masked data set may not be completely anonymized.
| 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 link:#sqlserver-temporal-values[temporal values].
| 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.
|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 _databaseName_._schemaName_._tableName_._columnName_.
|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 _databaseName_._schemaName_._tableName_._columnName_.
|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 `pass:[_]pass:[_]debezium.source.column.type`, `pass:[_]pass:[_]debezium.source.column.length` and `pass:[_]pass:[_]debezium.source.column.scale` is used to propagate the original type name and length (for variable-width types), respectively.
|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 `pass:[_]pass:[_]debezium.source.column.type`, `pass:[_]pass:[_]debezium.source.column.length` and `pass:[_]pass:[_]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.
Each item (regular expression) must match the fully-qualified `<fully-qualified table>:<a comma-separated list of columns>` representing the custom key. +
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.
|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. +
`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.
|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.
|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.
|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.
|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.
|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 link:#snapshots-sqlserver[snapshots]). +
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.
|`true` when connector configuration explicitly specifies the `key.converter` or `value.converter` parameters to use Avro, otherwise defaults to `false`.
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. +
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 {link-kafka-docs}.html#security_configclients[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.
Be sure to consult the {link-kafka-docs}.html[Kafka documentation] for all of the configuration properties for Kafka producers and consumers. (The SQL Server connector does use the {link-kafka-docs}.html#newconsumerconfigs[new consumer].)