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 Db2

This connector is currently in incubating state, i.e. exact semantics, configuration options etc. may change in future revisions, based on the feedback we receive. Please let us know if you encounter any problems.

Debezium’s Db2 Connector can monitor and record the row-level changes in the tables of a Db2 database. This connector is strongly inspired by the Debezium implementation of SQL Server, using a SQL based polling model of tables put into "capture mode". This uses the ASN libraries that come as a standard part of Db2 LUW and can be added to Db2 zOS.

In order to use ASN and hence this connector, you need to have a license for the IBM InfoSphere Data Replication (IIDR) product. It is not required that IIDR itself is installed.

The first time it connects to a Db2 database, it reads a consistent snapshot of all of the tables that are whitelisted (or not blacklisted depending on the mode of operation). Note that by default all tables in the database are snapshoted, NOT just those that are in capture mode.

When that snapshot is complete, the connector continuously streams the changes that were committed to the Db2 database for all whitelisted tables in capture mode. This 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 the ASN Capture/Apply agents that enable SQL Replication within DB2.

The capture agent generates ASN tables for all tables in CDC mode, these ASN tables can be queried via a SQL interface in order to read change events. Using this mechanism a Db2 ASN capture agent monitors all databases and tables that the user is interested in and stores the changes into specifically created ASN tables. The connector has been tested with Db2/Linux 11.5.0.0, but our expectation is that the model would work for Windows, AIX and zOS as well.

The database administrator must put the tables to be monitored into capture mode. For convenience and for automating testing we have written a User Defined Function (UDF) in C that can be compiled and then used to control the ASN agent, create the ASN schemas/tables and add/remove tables to capture. These utilities are described in debezium-connector-db2/src/test/docker/db2-cdc-docker/ but these are only for convenience and manually performing the same commands using db2 control commands would have the same effect.

The connector then produces a change event for every row-level insert, update, and delete operation that was published via the ASN SQL Tables, 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 administrator normally enables CDC in the middle of the life of a table. This means that the connector does not have the complete history of all changes that have been made to the table. Therefore, when the Db2 connector first connects to a particular Db2 database, it starts by performing a consistent snapshot of each of the whitelisted tables. After the connector completes the snapshot, it continues streaming changes from the exact point at which the snapshot was made for tables in capture mode. 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) of the CDC record for 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 will begin a new snapshot. Care needs to be taken with mixing multiple snapshot in the same topic as delete operations may be missed.

Setting up Db2

Before using the Db2 connector to monitor the changes committed on the Db2 database, first enable CDC on a monitored database.

We recommend using the UDF functions that we provide. However these are only for convenience and manually performing the same commands using db2 control commands would have the same effect.

We assume here that the contents of debezium-connector-db2/src/test/docker/db2-cdc-docker are available on the machine on which Db2 is running in $HOME/asncdctools/src and that the user has logged-in as the db2inst1 user.

First compile this function on the Db2 server, using the bldrtn C-compiler

$ cd $HOME/asncdctools/src
$ ./bldrtn asncdc

Start the database if not already running:

$ db2 start db <DB NAME>

We need to make sure that the metadata catalog can be read via JDBC

$ cd $HOME/sqllib/bnd
$ db2 bind db2schema.bnd blocking all grant public sqlerror continue

The database needs to have been recently backed-up, such that the ASN agents have a recent starting point to read from. Do the following if this is not the case. Note This will prune the data such that only the most recent version is available:

$ db2 backup db <DB NAME> to <BACK UP LOCATION>

/dev/null can be used for the location if the older versions of the data do not need to be retained.

$ db2 restart db <DB NAME>

Install the UDF:

$ db2 connect to <DB NAME>

We assume that the db2 tool is installed on the db2inst1 user

$ cp $HOME/asncdctools/src/asncdc $HOME/sqllib/function
$ chmod 777 $HOME/sqllib/function

Enable the UDF that allows the ASN capture agent to be started/stopped

$ db2 -tvmf $HOME/asncdctools/src/asncdc_UDF.sql

Create the ASN Control tables

$ db2 -tvmf $HOME/asncdctools/src/asncdctables.sql

Enable the UDF that allows us to add/remove tables to be captured

$ db2 -tvmf $HOME/asncdctools/src/asncdcaddremove.sql

Having done the above, we can use the UDFs to control ASN via SQL commands. Some of the UDFs expect a return value in which case we use the SQL VALUE statement to invoke them, while others are fire-and-forget in which case we use the SQL CALL statement.

First the ASN agent needs to be started:

VALUES ASNCDC.ASNCDCSERVICES('start','asncdc');

If the agent every needs to be stopped:

VALUES ASNCDC.ASNCDCSERVICES('stop','asncdc');

The status if the agent can be checked at any moment:

VALUES ASNCDC.ASNCDCSERVICES('status','asncdc');

A table MYTABLE in MYSCHEMA can be put into capture mode by doing:

CALL ASNCDC.ADDTABLE('MYSCHEMA', 'MYTABLE');

At table MYTABLE in MYSCHEMA can be removed from capture mode by doing:

CALL ASNCDC.REMOVETABLE('MYSCHEMA', 'MYTABLE');

After a table is added or removed, the user MUST reinitialize the ASN service:

VALUES ASNCDC.ASNCDCSERVICES('reinit','asncdc');

How the Db2 connector works

Snapshots

Db2 ASN is not designed to store the complete history of database changes. It is thus necessary that Debezium establishes the baseline of current table content and streams it to 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 table.

Each snapshot consists of the following steps:

  1. Determine the tables to be snapshoted from whitelist/blacklist

  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. Then write that event to the appropriate table-specific Kafka topic with the maxiumun LSN, i.e. all snapshot insert operations (i.e those with opcode='r') have the same LSN.

  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 requested 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 assures 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 the CDC is enabled or disabled for whitelisted source table during the runtime and modify its behavior.

Topic names

The Db2 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 databaseName.schemaName.tableName, where databaseName 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.

Unlike for SQL Server, it is only possible for a connector to streams changes from one Db2 database.

For example, consider a Db2 installation with an mydatabase database that contains four tables: PRODUCTS, PRODUCTS_ON_HAND, CUSTOMERS, and ORDERS in schema MYSCHEMA then the connector would produce events on these four Kafka topics:

  • mydatabase.MYSCHEMA.PRODUCTS

  • mydatabase.MYSCHEMA.PRODUCTS_ON_HAND

  • mydatabase.MYSCHEMA.CUSTOMERS

  • mydatabase.MYSCHEMA.ORDERS

Events

All data change events produced by the Db2 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 Topic names).

The Debezium Db2 connector ensures that all Kafka Connect schema names are 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 database 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. In addition, note that databases, schemas and tables can be case sensitive in Db2 meaning that different tables maybe mapped to the same Kafka topic.

Debezium and Kafka Connect are designed around continuous streams of event messages, and the structure of these events may change over time. 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

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 mydatabase database’s schema MYSCHEMA:

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 database.server.name configuration property is then mydatabase. Every change event for the customers table while it has this definition will feature the same key structure, which in JSON looks like this:

{
    "schema": {
        "type": "struct",
        "fields": [
            {
                "type": "int32",
                "optional": false,
                "field": "ID"
            }
        ],
        "optional": false,
        "name": "mydatabase.MYSCHEMA.CUSTOMERS.Key"
    },
    "payload": {
        "ID": 1004
    }
}

The schema portion of the key contains a Kafka Connect schema describing what is in the key portion. In this case, it means that the payload value is not optional, is a structure defined by a schema named mydatabase.MYSCHEMA.CUSTOMERS.Key, and has one required field named id of type int32. If you look at the value of the key’s payload field, you can see that it is indeed a structure (which in JSON is just an object) with a single id field, whose value is 1004.

Therefore, you can interpret this key as describing the row in the MYSCHEMA.CUSTOMERS table (output from the connector reading from database mydatabase) whose id primary key column had a value of 1004.

Change Event Values

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 Db2 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 Db2 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 will be described by the mydatabase.MYSCHEMA.CUSTOMERS.Value Kafka Connect schema, which the connector reading from mydatabase uses for all rows in the MYSCHEMA.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 mydatabase.MYSCHEMA.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 Db2 contains these fields: the Debezium 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 read from the the source database by the connector.

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

Create events

Let’s look at what a create event value might look like for our customers table:

{
  "schema": {
    "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": "mydatabase.MYSCHEMA.CUSTOMERS.Value",
        "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": "mydatabase.MYSCHEMA.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_ms"
          },
          {
            "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"
          },
        ],
        "optional": false,
        "name": "io.debezium.connector.db2.Source",
        "field": "source"
      },
      {
        "type": "string",
        "optional": false,
        "field": "op"
      },
      {
        "type": "int64",
        "optional": true,
        "field": "ts_ms"
      }
    ],
    "optional": false,
    "name": "mydatabase.MYSCHEMA.CUSTOMERS.Envelope"
  },
  "payload": {
    "before": null,
    "after": {
      "ID": 1005,
      "FIRST_NAME": "john",
      "LAST_NAME": "doe",
      "EMAIL": "john.doe@example.org"
    },
    "source": {
      "version": "1.1.2.Final",
      "connector": "db2",
      "name": "myconnector",
      "ts_ms": 1559729468470,
      "snapshot": false,
      "db": "mydatabase",
      "schema": "MYSCHEMA",
      "table": "CUSTOMERS",
      "change_lsn": "00000027:00000758:0003",
      "commit_lsn": "00000027:00000758:0005",
    },
    "op": "c",
    "ts_ms": 1559729471739
  }
}

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

It may appear that the JSON representations of the events are much larger than the rows they describe. This is true, because the JSON representation must include the schema and the payload portions of the message. It is possible and even recommended to use the Avro Converter to dramatically decrease the size of the actual messages written to the Kafka topics.

Update events

The value of an update change event on this table will actually have the exact same schema, and its payload will be structured the same but will hold different values. Here’s an example:

{
  "schema": { ... },
  "payload": {
    "before": {
      "ID": 1005,
      "FIRST_NAME": "john",
      "LAST_NAME": "doe",
      "EMAIL": "john.doe@example.org"
    },
    "after": {
      "ID": 1005,
      "FIRST_NAME": "john",
      "LAST_NAME": "doe",
      "EMAIL": "noreply@example.org"
    },
    "source": {
      "version": "1.1.2.Final",
      "connector": "db2",
      "name": "myconnector",
      "ts_ms": 1559729995937,
      "snapshot": false,
      "db": "mydatabase",
      "schema": "MYSCHEMA",
      "table": "CUSTOMERS",
      "change_lsn": "00000027:00000ac0:0002",
      "commit_lsn": "00000027:00000ac0:0007",
    },
    "op": "u",
    "ts_ms": 1559729998706
  }
}

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.

  • The ts_ms shows the timestamp that Debezium processed this event.

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 Db2’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 Db2 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 Debezium will output three events: a DELETE event and a tombstone event with the old key for the row, followed by an INSERT event with the new key for the row.

Delete events

So far we’ve seen samples of create and update events. Now, let’s look at the value of a delete event for the same table. Once again, the schema portion of the value will be exactly the same as with the create and update events:

{
  "schema": { ... },
  },
  "payload": {
    "before": {
      "ID": 1005,
      "FIRST_NAME": "john",
      "LAST_NAME": "doe",
      "EMAIL": "noreply@example.org"
    },
    "after": null,
    "source": {
      "version": "1.1.2.Final",
      "connector": "db2",
      "name": "myconnector",
      "ts_ms": 1559730445243,
      "snapshot": false,
      "db": "mydatabase",
      "schema": "MYSCHEMA",
      "table": "CUSTOMERS",
      "change_lsn": "00000027:00000db0:0005",
      "commit_lsn": "00000027:00000db0:0007"
    },
    "op": "d",
    "ts_ms": 1559730450205
  }
}

If we look at the payload portion, we see a number of differences compared with the create or update event payloads:

  • The op field value is now d, signifying that this row was deleted

  • The before field now has the state of the row that was deleted with the database commit.

  • The after field is null, signifying that the row no longer exists

  • The source field structure has many of the same values as before, except the ts_ms, commit_lsn and change_lsn fields have changed

  • The ts_ms shows the timestamp that Debezium processed this event.

This event gives a consumer all kinds of information that it can use to process the removal of this row.

The Db2 connector’s events are designed to work with Kafka log compaction, which allows for the removal of some older messages as long as at least the most recent message for every key is kept. This allows Kafka to reclaim storage space while ensuring the topic contains a complete dataset and can be used for reloading key-based state.

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, Debezium’s Db2 connector always follows the delete event with a special tombstone event that has the same key but 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 Db2, it is necessary to work in co-operation with a database administrator in order to ensure the Debezium connector continues to produce data change events when the schema is updated.

As was already mentioned before, Debezium uses Db2’s change data capture functionality. This means that Db2 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 Debezium connector itself but must be executed by an administrator 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. We note here some of the likely effects:

  • If a column name is changed then the old column will continue to be used by the ASN capture service and therefore the new name will not appear in Debezium. If Debezium is restarted then the new name will appear.

  • To Be Completed

It is recommend that if the structure of a source change is changed, that we: - that we mark the tables as inactive on the ASN register table - reinit the ASN capture service (see the UDFs) - update the ASN representation of the table (manual task) - mark the table as active - reinit the ASN capture service again (see the UDFs)

Cold schema update

This is the safest procedure but might not be feasible for applications with high-availability requirements. The administrator 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 Debezium connector

  4. Apply all changes to the source table schema

  5. Mark the tables as INACTIVE on the ASN register table and reinit the ASN capture service (see the UDFs)

  6. Remove the old structure table from ASN

  7. Add the new structure table to ASN

  8. Mark the tables as ACTIVE on the ASN register table and reinit the ASN capture service (see the UDFs)

  9. Resume the application

  10. Start Debezium connector

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. First we consider an incremental change to the source, e.g. adding a new column to the end:

  1. lock the source table to change

  2. Mark the tables as INACTIVE on the ASN register table and reinit the ASN capture service (see the UDFs)34. Apply all changes to the source table schema

  3. Apply all changes to the ASN table schema

  4. Mark the tables as ACTIVE on the ASN register table and reinit the ASN capture service (see the UDFs)

Now we consider an non-incremental change to the source, e.g. adding a new column in the middle:

  1. lock the source table to change

  2. Mark the tables as INACTIVE on the ASN register table and reinit the ASN capture service (see the UDFs)

  3. export the data of the source table to change

  4. truncate the source table

  5. alter the source table

  6. LOAD the exported data into the altered source table

  7. export the data of the ASN table to change

  8. truncate the ASN table

  9. alter the ASN table

  10. LOAD the exported data into the altered ASN table

  11. Mark the tables as INACTIVE on the ASN register table and reinit the ASN capture service (see the UDFs)

Example

To Be Done

Data types

A summary of Db2’s data types are described in Data Types.

As described above, the Db2 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 Db2 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.

Db2 Data Type Literal type (schema type) Semantic type (schema name) Notes

BOOLEAN

BOOLEAN

n/a

BIGINT

INT64

n/a

BINARY

BYTES

n/a

BLOB

BYTES

n/a

CHAR[(N)]

STRING

n/a

CLOB

STRING

n/a

DATE

INT32

io.debezium.time.Date

A string representation of a timestamp without timezone information

DECFLOAT

BYTES

org.apache.kafka.connect.data.Decimal

DECIMAL

BYTES

org.apache.kafka.connect.data.Decimal

DBCLOB

STRING

n/a

DOUBLE

FLOAT64

n/a

INTEGER

INT32

n/a

REAL

FLOAT32

n/a

SMALLINT

INT16

n/a

TIME

INT32

io.debezium.time.Time

A string representation of a times without timezone information

TIMESTAMP

INT64

io.debezium.time.MicroTimestamp

A string representation of a timestamp without timezone information

VARBINARY

BYTES

n/a

VARCHAR[(N)]

STRING

n/a

VARGRAPHIC

STRING

n/a

XML

STRING

io.debezium.data.Xml

Contains the string representation of a XML document

Other data type mappings are described in the following sections.

If present, a column’s default value will be 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 Db2’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:

Db2 Data Type Literal type (schema type) Semantic type (schema name) Notes

DATE

INT32

io.debezium.time.Date

Represents the number of days since 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 epoch, and does not include timezone information.

SMALLDATETIME

INT64

io.debezium.time.Timestamp

Represents the number of milliseconds past 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 epoch, and does not include timezone information.

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

INT64

io.debezium.time.MicroTimestamp

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

DATETIME2(7)

INT64

io.debezium.time.NanoTimestamp

Represents the number of nanoseconds past 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 Db2 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:

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

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" will be 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

Db2 Data Type Literal type (schema type) Semantic type (schema name) Notes

NUMERIC[(P[,S])]

BYTES

org.apache.kafka.connect.data.Decimal

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

DECIMAL[(P[,S])]

BYTES

org.apache.kafka.connect.data.Decimal

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

SMALLMONEY

BYTES

org.apache.kafka.connect.data.Decimal

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

MONEY

BYTES

org.apache.kafka.connect.data.Decimal

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

Deploying a connector

If you’ve already installed Zookeeper, Kafka, and Kafka Connect, then using Debezium’s Db2` connector is easy. First download the connector’s plug-in archive, extract it, and add the contained directory to Kafka Connect’s plugin.path by using the plugin.path configuration property.

In addition, due to licensing reasons you need to separately obtain the JDBC driver for Db2. Add the JDBC driver JAR to the directory with the Debezium Db2 connector JARs. Restart your Kafka Connect process to pick up the new connector.

If immutable containers are your thing, then check out Debezium’s Container images for Zookeeper, Kafka and Kafka Connect with the Db2 connector already pre-installed and ready to go. You can even run Debezium on OpenShift.

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

  1. enable the CDC on Db2 to publish the CDC events in the database

  2. create a configuration file for the Db2 Connector and use the Kafka Connect REST API to add that connector to your Kafka Connect cluster.

When the connector starts, it will grab a consistent snapshot of the schemas in your Db2 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.

Example configuration

Using the Db2 connector is straightforward. Here is an example of the configuration for a connector instance that monitors a Db2 server at port 50000 on 192.168.99.100, which we logically name fullfillment:

{
  "name": "db2-connector",  (1)
  "config": {
    "connector.class": "io.debezium.connector.db2.Db2Connector", (2)
    "database.hostname": "192.168.99.100", (3)
    "database.port": "50000", (4)
    "database.user": "db2inst1", (5)
    "database.password": "Password!", (6)
    "database.dbname": "mydatabase", (7)
    "database.server.name": "fullfillment", (8)
    "table.whitelist": "MYSCHEMA.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 Db2 connector class.
3 The address of the Db2 instance.
4 The port number of the Db2 instance.
5 The name of the Db2 user
6 The password for the Db2 user
7 The name of the database to capture changes from
8 The logical name of the Db2 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 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 Db2 database, read the transaction log, and record events to Kafka topics.

Monitoring

The Debezium Db2 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.db2:type=connector-metrics,context=snapshot,server=<database.server.name>.

Attribute Name Type Description

LastEvent

string

The last snapshot event that the connector has read.

MilliSecondsSinceLastEvent

long

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

TotalNumberOfEventsSeen

long

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

NumberOfEventsFiltered

long

The number of events that have been filtered by whitelist or blacklist filtering rules configured on the connector.

MonitoredTables

string[]

The list of tables that are monitored by the connector.

QueueTotalCapcity

int

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

QueueRemainingCapcity

int

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

TotalTableCount

int

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

RemainingTableCount

int

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

SnapshotRunning

boolean

Whether the snapshot was started.

SnapshotAborted

boolean

Whether the snapshot was aborted.

SnapshotCompleted

boolean

Whether the snapshot completed.

SnapshotDurationInSeconds

long

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

RowsScanned

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.db2:type=connector-metrics,context=streaming,server=<database.server.name>.

Attribute Name Type Description

LastEvent

string

The last streaming event that the connector has read.

MilliSecondsSinceLastEvent

long

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

TotalNumberOfEventsSeen

long

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

NumberOfEventsFiltered

long

The number of events that have been filtered by whitelist or blacklist filtering rules configured on the connector.

MonitoredTables

string[]

The list of tables that are monitored by the connector.

QueueTotalCapcity

int

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

QueueRemainingCapcity

int

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

Connected

boolean

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

MilliSecondsBehindSource

long

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

NumberOfCommittedTransactions

long

The number of processed transactions that were committed.

SourceEventPosition

map<string, string>

The coordinates of the last received event.

LastTransactionId

string

Transaction identifier of the last processed transaction.

Schema History Metrics

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

Attribute Name Type Description

Status

string

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

RecoveryStartTime

long

The time in epoch seconds at what recovery has started.

ChangesRecovered

long

The number of changes that were read during recovery phase.

ChangesApplied

long

The total number of schema changes applie during recovery and runtime.

MilliSecondsSinceLastRecoveredChange

long

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

MilliSecondsSinceLastAppliedChange

long

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

LastRecoveredChange

string

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

LastAppliedChange

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.db2.Db2Connector for the Db2 connector.

1

The maximum number of tasks that should be created for this connector. The Db2 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 Db2 database server.

50000

Integer port number of the Db2 database server.

Username to use when connecting to the Db2 database server.

Password to use when connecting to the Db2 database server.

The name of the Db2 database from which to stream the changes

Logical name that identifies and provides a namespace for the particular Db2 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 will be 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 will be 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 will be monitored. Each identifier is of the form schemaName.tableName. May not be used with table.whitelist.

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 blacklisted from the value.

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

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 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 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 Db2 data types for the list of Db2-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. Supported values are initial (will take 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) and schema_only (will take a snapshot of the structure of captured tables only; useful if only changes happening from now onwards should be propagated to topics). Once the snapshot is complete, the connector will continue reading change events from the database’s redo logs.

repeatable_read

Mode to control which transaction isolation level is used and how long the connector locks the monitored tables. There are four possible values: read_uncommitted, read_committed, repeatable_read, 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 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 the exclusive mode guarantees full consistency, that is, initial snapshot and streaming logs constitute a linear history. In case of repeatable_read and read_committed modes, it might happen that, for instance, a record added appears twice - once in initial snapshot and once in streaming phase. Nonetheless, that consistency level should do for data mirroring. For read_uncommitted there are no data consistency guarantees at all (some data might be lost or corrupted).

fail

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

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

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

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

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

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.

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 Db2 connector does use the new consumer.)