DBZ-1419 Document adaptive/connect mode for SQL Server

This commit is contained in:
Chris Cranford 2019-09-05 17:37:58 -04:00 committed by Gunnar Morling
parent d56eb80fd2
commit 2118b8a1d2

View File

@ -760,8 +760,11 @@ 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 xref:configuration/avro.advoc[using Avro] as serialization format together with the Confluent schema registry.
[[temporal-values]]
==== 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:
[cols="20%a,15%a,30%a,35%a",width=150,options="header,footer",role="table table-bordered table-striped"]
|=======================
|SQL Server Data Type
@ -816,6 +819,42 @@ Passing the default value helps though with satisfying the compatibility rules w
|=======================
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:
[cols="20%a,15%a,30%a,35%a",width=150,options="header,footer",role="table table-bordered table-striped"]
|=======================
|SQL Server Data Type
|Literal type (schema type)
|Semantic type (schema name)
|Notes
|`DATE`
|`INT32`
|`org.apache.kafka.connect.data.Date`
| Represents the number of days since 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 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.
|=======================
[[timestamp-values]]
===== Timestamp values
@ -829,7 +868,7 @@ Note that the timezone of the JVM running Kafka Connect and Debezium does not af
[cols="15%a,15%a,35%a,35%a",width=100,options="header,footer",role="table table-bordered table-striped"]
|=======================
|PostgreSQL Data Type
|SQL Server Data Type
|Literal type (schema type)
|Semantic type (schema name)
|Notes
@ -1175,6 +1214,10 @@ The following configuration properties are _required_ unless a default value is
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.
|`time.precision.mode`
|`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 link:#temporal-values[temporal values].
|`tombstones.on.delete` +
0.10.0 and later
|`true`