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

Debezium Connector for Oracle

Table of Contents

Overview

Debezium’s Oracle connector captures and records row-level changes that occur in databases on an Oracle server, including tables that are added while the connector is running. You can configure the connector to emit change events for specific subsets of schemas and tables, or to ignore, mask, or truncate values in specific columns.

For information about the Oracle Database versions that are compatible with this connector, see the Debezium release overview.

Debezium ingests change events from Oracle by using the native LogMiner database package or the XStream API. While the connector might work with a variety of Oracle versions and editions, only Oracle EE 12 and 19 have been tested.

How the Oracle connector works

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

Snapshots

Typically, the redo logs on an Oracle server are configured to not retain the complete history of the database. As a result, the Debezium Oracle connector cannot retrieve the entire history of the database from the logs. To enable the connector to establish a baseline for the current state of the database, the first time that the connector starts, it performs an initial consistent snapshot of the database.

You can customize the way that the connector creates snapshots by setting the value of the snapshot.mode connector configuration property. By default, the connector’s snapshot mode is set to initial.

Default connector workflow for creating an initial snapshot

When the snapshot mode is set to the default, the connector completes the following tasks to create a snapshot:

  1. Determines the tables to be captured.

  2. Obtains a ROW SHARE MODE lock on each of the monitored tables to prevent structural changes from occurring during creation of the snapshot. Debezium holds the locks for only a short time.

  3. Reads the current system change number (SCN) position from the server’s redo log.

  4. Captures the structure of all relevant tables.

  5. Releases the locks obtained in Step 2.

  6. Scans all of the relevant database tables and schemas as valid at the SCN position that was read in Step 3 (SELECT * FROM …​ AS OF SCN 123), generates a READ event for each row, and then writes the event records to the table-specific Kafka topic.

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

After the snapshot process begins, if the process is interrupted due to connector failure, rebalancing, or other reasons, the process restarts after the connector restarts. After the connector completes the initial snapshot, it continues streaming from the position that it read in Step 3 so that it does not miss any updates. If the connector stops again for any reason, after it restarts, it resumes streaming changes from where it previously left off.

Table 1. Settings for snapshot.mode connector configuration property
Setting Description

initial

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

initial_only

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

schema_only

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

schema_only_recovery

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

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

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

Ad hoc snapshots

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

However, in some situations the data that the connector obtained during the initial snapshot might become stale, lost, or incomplete. To provide a mechanism for recapturing table data, Debezium includes an option to perform ad hoc snapshots. The following changes in a database might be cause for performing an ad hoc snapshot:

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

  • Kafka topics are deleted and must be rebuilt.

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

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

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

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

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

Table 2. Example of an ad hoc execute-snapshot signal record
Field Default Value

type

incremental

Specifies the type of snapshot that you want to run.
Setting the type is optional. Currently, you can request only incremental snapshots.

data-collections

N/A

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

Triggering an ad hoc snapshot

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

Currently, the execute-snapshot action type triggers incremental snapshots only. For more information, see Incremental snapshots.

Incremental snapshots

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

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

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

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

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

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

Incremental snapshot process

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

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

How Debezium resolves collisions among records with the same primary key

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

Snapshot window

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

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

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

The connector repeats the process for each snapshot chunk.

Triggering an incremental snapshot

Currently, the only way to initiate an incremental snapshot is to send an ad hoc snapshot signal to the signaling table on the source database. You submit signals to the table as SQL INSERT queries. After Debezium detects the change in the signaling table, it reads the signal, and runs the requested snapshot operation.

The query that you submit specifies the tables to include in the snapshot, and, optionally, specifies the kind of snapshot operation. Currently, the only valid option for snapshots operations is the default value, incremental.

To specify the tables to include in the snapshot, provide a data-collections array that lists the tables, for example,
{"data-collections": ["public.MyFirstTable", "public.MySecondTable"]}

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

Prerequisites
  • Signaling is enabled.

    • A signaling data collection exists on the source database and the connector is configured to capture it.

    • The signaling data collection is specified in the signal.data.collection property.

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

    INSERT INTO _<signalTable>_ (id, type, data) VALUES (_'<id>'_, _'<snapshotType>'_, '{"data-collections": ["_<tableName>_","_<tableName>_"],"type":"_<snapshotType>_"}');

    For example,

    INSERT INTO myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["schema1.table1", "schema2.table2"],"type":"incremental"}');

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

    The following table describes the these parameters:

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

    myschema.debezium_signal

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

    ad-hoc-1

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

    execute-snapshot

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

    data-collections

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

    incremental

    An optional type component of the data field of a signal that specifies the kind of snapshot operation to run.
    Currently, the only valid option is the default value, incremental.
    Specifying a type value in the SQL query that you submit to the signaling table is optional.
    If you do not specify a value, the connector runs an incremental snapshot.

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

Example: Incremental snapshot event message
{
    "before":null,
    "after": {
        "pk":"1",
        "value":"New data"
    },
    "source": {
        ...
        "snapshot":"incremental" (1)
    },
    "op":"r", (2)
    "ts_ms":"1620393591654",
    "transaction":null
}
Item Field name Description

1

snapshot

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

2

op

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

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

Topic names

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

serverName.schemaName.tableName

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

serverName

The logical name of the server as specified by the database.server.name connector configuration property.

schemaName

The name of the schema in which the operation occurred.

tableName

The name of the table in which the operation occurred.

For example, if fulfillment is the server name, inventory is the schema name, and the database contains tables with the names orders, customers, and products, the Debezium Oracle connector emits events to the following Kafka topics, one for each table in the database:

fulfillment.inventory.orders
fulfillment.inventory.customers
fulfillment.inventory.products

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

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

Schema change topic

You can configure a Debezium Oracle connector to produce schema change events that describe structural changes that are applied to captured tables in the database. The connector writes schema change events to a Kafka topic named <serverName>, where serverName is the logical server name that is specified in the database.server.name configuration property.

Debezium emits a new message to this topic whenever it streams data from a new table.

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

ddl

Provides the SQL CREATE, ALTER, or DROP statement that results in the schema change.

databaseName

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

tableChanges

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

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

Never partition the database history topic. For the database history topic to function correctly, it must maintain a consistent, global order of the event records that the connector emits to it.

To ensure that the topic is not split among partitions, set the partition count for the topic by using one of the following methods:

  • If you create the database history topic manually, specify a partition count of 1.

  • If you use the Apache Kafka broker to create the database history topic automatically, the topic is created, set the value of the Kafka num.partitions configuration option to 1.

The schema change topic message format is in an incubating state and might change without notice.

Debezium emits a new message to this topic whenever it streams data from a new table, or when the structure of the table is altered.

Following a change in table structure, you must follow (the schema evolution procedure).

Example: Message emitted to the Oracle connector schema change topic

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

{
  "schema": {
  ...
  },
  "payload": {
    "source": {
      "version": "2.0.0.Alpha1",
      "connector": "oracle",
      "name": "server1",
      "ts_ms": 1588252618953,
      "snapshot": "true",
      "db": "ORCLPDB1",
      "schema": "DEBEZIUM",
      "table": "CUSTOMERS",
      "txId" : null,
      "scn" : "1513734",
      "commit_scn": "1513734",
      "lcr_position" : null
    },
    "ts_ms": 1588252618953, (1)
    "databaseName": "ORCLPDB1", (2)
    "schemaName": "DEBEZIUM", //
    "ddl": "CREATE TABLE \"DEBEZIUM\".\"CUSTOMERS\" \n   (    \"ID\" NUMBER(9,0) NOT NULL ENABLE, \n    \"FIRST_NAME\" VARCHAR2(255), \n    \"LAST_NAME" VARCHAR2(255), \n    \"EMAIL\" VARCHAR2(255), \n     PRIMARY KEY (\"ID\") ENABLE, \n     SUPPLEMENTAL LOG DATA (ALL) COLUMNS\n   ) SEGMENT CREATION IMMEDIATE \n  PCTFREE 10 PCTUSED 40 INITRANS 1 MAXTRANS 255 \n NOCOMPRESS LOGGING\n  STORAGE(INITIAL 65536 NEXT 1048576 MINEXTENTS 1 MAXEXTENTS 2147483645\n  PCTINCREASE 0 FREELISTS 1 FREELIST GROUPS 1\n  BUFFER_POOL DEFAULT FLASH_CACHE DEFAULT CELL_FLASH_CACHE DEFAULT)\n  TABLESPACE \"USERS\" ", (3)
    "tableChanges": [ (4)
      {
        "type": "CREATE", (5)
        "id": "\"ORCLPDB1\".\"DEBEZIUM\".\"CUSTOMERS\"", (6)
        "table": { (7)
          "defaultCharsetName": null,
          "primaryKeyColumnNames": [ (8)
            "ID"
          ],
          "columns": [ (9)
            {
              "name": "ID",
              "jdbcType": 2,
              "nativeType": null,
              "typeName": "NUMBER",
              "typeExpression": "NUMBER",
              "charsetName": null,
              "length": 9,
              "scale": 0,
              "position": 1,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            },
            {
              "name": "FIRST_NAME",
              "jdbcType": 12,
              "nativeType": null,
              "typeName": "VARCHAR2",
              "typeExpression": "VARCHAR2",
              "charsetName": null,
              "length": 255,
              "scale": null,
              "position": 2,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            },
            {
              "name": "LAST_NAME",
              "jdbcType": 12,
              "nativeType": null,
              "typeName": "VARCHAR2",
              "typeExpression": "VARCHAR2",
              "charsetName": null,
              "length": 255,
              "scale": null,
              "position": 3,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            },
            {
              "name": "EMAIL",
              "jdbcType": 12,
              "nativeType": null,
              "typeName": "VARCHAR2",
              "typeExpression": "VARCHAR2",
              "charsetName": null,
              "length": 255,
              "scale": null,
              "position": 4,
              "optional": false,
              "autoIncremented": false,
              "generated": false
            }
          ]
        }
      }
    ]
  }
}
Table 4. Descriptions of fields in messages emitted to the schema change topic
Item Field name Description

1

ts_ms

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

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

2

databaseName
schemaName

Identifies the database and the schema that contains the change.

3

ddl

This field contains the DDL that is responsible for the schema change.

4

tableChanges

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

5

type

Describes the kind of change. The type can be set to one of the following values:

CREATE

Table created.

ALTER

Table modified.

DROP

Table deleted.

6

id

Full identifier of the table that was created, altered, or dropped. In the case of a table rename, this identifier is a concatenation of <old>,<new> table names.

7

table

Represents table metadata after the applied change.

8

primaryKeyColumnNames

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

9

columns

Metadata for each column in the changed table.

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

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

Transaction Metadata

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

Limits on when Debezium receives transaction metadata

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

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

status

BEGIN or END

id

String representation of 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 elements that indicates number of events that the connector emits for changes that originate from a data collection.

The following example shows a typical transaction boundary message:

Example: Oracle connector transaction boundary event
{
  "status": "BEGIN",
  "id": "5.6.641",
  "event_count": null,
  "data_collections": null
}

{
  "status": "END",
  "id": "5.6.641",
  "event_count": 2,
  "data_collections": [
    {
      "data_collection": "ORCLPDB1.DEBEZIUM.CUSTOMER",
      "event_count": 1
    },
    {
      "data_collection": "ORCLPDB1.DEBEZIUM.ORDER",
      "event_count": 1
    }
  ]
}

Unless overridden via the transaction.topic option, the connector emits transaction events to the <database.server.name>.transaction topic.

Change data event enrichment

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

id

String representation of unique transaction identifier.

total_order

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

data_collection_order

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

The following example shows a typical transaction event message:

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

Event buffering

Oracle writes all changes to the redo logs in the order in which they occur, including changes that are later discarded by a rollback. As a result, concurrent changes from separate transactions are intertwined. When the connector first reads the stream of changes, because it cannot immediately determine which changes are committed or rolled back, it temporarily stores the change events in an internal buffer. After a change is committed, the connector writes the change event from the buffer to Kafka. The connector drops change events that are discarded by a rollback.

You can configure the buffering mechanism that the connector uses by setting the property log.mining.buffer.type.

Heap

The default buffer type is configured using memory. Under the default memory setting, the connector uses the heap memory of the JVM process to allocate and manage buffered event records. If you use the memory buffer setting, be sure that the amount of memory that you allocate to the Java process can accommodate long-running and large transactions in your environment.

Infinispan

The Debezium Oracle connector can also be configured to use Infinispan as its cache provider, supporting cache stores both locally with embedded mode or remotely on a server cluster. In order to use Infinispan, the log.mining.buffer.type must be configured using either infinispan_embedded or infinispan_remote.

In order to allow flexibility with Infinispan cache configurations, the connector expects a series of cache configuration properties to be supplied when using Infinispan to buffer event data. See the configuration properties in the log.mining.buffer.infinispan.cache namespace. The contents of these configuration properties depend on whether the connector is to integrate with a remote Infinispan cluster or to use the embedded engine.

For example, the following illustrates what an embedded configuration would look like for the transaction cache property when using Infinispan in embedded mode:

<local-cache name="transactions">
  <persistence passivation="false">
    <file-store fetch-state="true" read-only="false" preload="true" shared="false" segmented="false" path="./data" />
  </persistence>
</local-cache>

Looking at the configuration in-depth, the cache is configured to be persistent. All caches should be configured this way to avoid loss of transaction events across connector restarts if a transaction is in-progress. Additionally, the location where the cache is kept is defined by the path attribute and this should be a shared location accessible all possible runtime environments.

When supplying XML configuration as a JSON connector property value, line breaks must be omitted or replaced with a \n character.

Another example, the following illustrates the same cache configured with an Infinispan cluster:

<distributed-cache name="transactions" statistics="true">
  <encoding media-type="application/x-protostream" />
  <persistence passivation="false">
    <file-store fetch-state="true" read-only="false" preload="true" shared="false" segmented="false" />
  </persistence>
</distributed-cache>

Just like the embedded local-cache configuration from the previous example, this configuration is also defined to be persistent. All caches should be configured this way to avoid loss of transaction events across connector restarts if a transaction is in-progress.

However, there are a few differences with noting. First, the cache is defined as a distributed cache rather than a local-cache. Secondly, the cache is defined to use the application/x-protostream encoding, which is required for all Debezium caches. And lastly, no path attribute is necessary on the file store definition since the Infinispan cluster will handle this automatically.

The Infinispan buffer type is considered incubating; the cache formats may change between versions and may require a re-snapshot. The migration notes will indicate whether this is needed.

Additionally, when removing a Debezium Oracle connector that uses the Infinispan buffer, the persisted cache files are not removed from disk automatically. If the same buffer location will be used by a new connector deployment, the files should be removed manually before deploying the new connector.

Infinispan Hotrod client integration

The Debezium Oracle connector utilizes the Hotrod client to communicate with the Infinispan cluster. Any connector property that is prefixed with log.mining.buffer.infinispan.client. will be passed directly to the Hotrod client using the infinispan.client. namespace, allowing for complete customization of how the client is to interact with the cluster.

There is at least one required configuration property that must be supplied when using this Infinspan mode:

log.mining.buffer.infinispan.client.hotrod.server_list

Specifies the list of Infinispan server hostname and port combinations, using <hostname>:<port> format.

SCN gap detection

When the Debezium Oracle connector is configured to use LogMiner, it collects change events from Oracle by using a start and end range that is based on system change numbers (SCNs). The connector manages this range automatically, increasing or decreasing the range depending on whether the connector is able to stream changes in near real-time, or must process a backlog of changes due to the volume of large or bulk transactions in the database.

Under certain circumstances, the Oracle database advances the SCN by an unusually high amount, rather than increasing the SCN value at a constant rate. Such a jump in the SCN value can occur because of the way that a particular integration interacts with the database, or as a result of events such as hot backups.

The Debezium Oracle connector relies on the following configuration properties to detect the SCN gap and adjust the mining range.

log.mining.scn.gap.detection.gap.size.min

Specifies the minimum gap size.

log.mining.scn.gap.detection.time.interval.max.ms

Specifies the maximum time interval.

The connector first compares the difference in the number of changes between the current SCN and the highest SCN in the current mining range. If the difference between the current SCN value and the highest SCN value is greater than the minimum gap size, then the connector has potentially detected a SCN gap. To confirm whether a gap exists, the connector next compares the timestamps of the current SCN and the SCN at the end of the previous mining range. If the difference between the timestamps is less than the maximum time interval, then the existence of an SCN gap is confirmed.

When an SCN gap occurs, the Debezium connector automatically uses the current SCN as the end point for the range of the current mining session. This allows the connector to quickly catch up to the real-time events without mining smaller ranges in between that return no changes because the SCN value was increased by an unexpectedly large number. When the connector performs the preceding steps in response to an SCN gap, it ignores the value that is specified by the log.mining.batch.size.max property. After the connector finishes the mining session and catches back up to real-time events, it resumes enforcement of the maximum log mining batch size.

SCN gap detection is available only if the large SCN increment occurs while the connector is running and processing near real-time events.

Low change frequency offset management

The Debezium Oracle connector tracks system change numbers in the connector offsets so that when the connector is restarted, it can begin where it left off. These offsets are part of each emitted change event; however, when the frequency of database changes are low (every few hours or days), the offsets can become stale and prevent the connector from successfully restarting if the system change number is no longer available in the transaction logs.

For connectors that use non-CDB mode to connect to Oracle, you can enable heartbeat.interval.ms to force the connector to emit a heartbeat event at regular intervals so that offsets remain synchronized.

For connectors that use CDB mode to connect to Oracle, maintaining synchronization is more complicated. Not only must you set heartbeat.interval.ms, but it’s also necessary to set heartbeat.action.query. Specifying both properties is required, because in CDB mode, the connector specifically tracks changes inside the PDB only. A supplementary mechanism is needed to trigger change events from within the pluggable database. At regular intervals, the heartbeat action query causes the connector to insert a new table row, or update an existing row in the pluggable database. Debezium detects the table changes and emits change events for them, ensuring that offsets remain synchronized, even in pluggable databases that process changes infrequently.

For the connector to use the heartbeat.action.query with tables that are not owned by the connector user account, you must grant the connector user permission to run the necessary INSERT or UPDATE queries on those tables.

Data change events

Every data change event that the Oracle connector emits has a key and a value. The structures of the key and value depend on the table from which the change events originate. For information about how Debezium constructs topic names, see Topic names.

The Debezium Oracle connector ensures that all Kafka Connect schema names are valid Avro schema names. This means that the logical server name must start with alphabetic characters or an underscore ([a-z,A-Z,_]), and the remaining characters in the logical server name and all characters in the schema and table names must be alphanumeric characters or an underscore ([a-z,A-Z,0-9,\_]). The connector automatically replaces invalid characters with an underscore character.

Unexpected naming conflicts can result when the only distinguishing characters between multiple logical server names, schema names, or table names are not valid characters, and those characters are replaced with underscores.

Debezium and Kafka Connect are designed around continuous streams of event messages. However, the structure of these events might change over time, which can be difficult for topic consumers to handle. To facilitate the processing of mutable event structures, each event in Kafka Connect is 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.

Changes that are performed by the SYS or SYSTEM user accounts are not captured by the connector.

Change event keys

For each changed table, the change event key is structured such that a field exists for each column in the primary key (or unique key constraint) of the table at the time when the event is created.

For example, a customers table that is defined in the inventory database schema, might have the following change event key:

CREATE TABLE customers (
  id NUMBER(9) GENERATED BY DEFAULT ON NULL AS IDENTITY (START WITH 1001) NOT NULL PRIMARY KEY,
  first_name VARCHAR2(255) NOT NULL,
  last_name VARCHAR2(255) NOT NULL,
  email VARCHAR2(255) NOT NULL UNIQUE
);

If the value of the <database.server.name>.transaction configuration property is set to server1, the JSON representation for every change event that occurs in the customers table in the database features the following key structure:

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

The schema portion of the key contains a Kafka Connect schema that describes the content of the key portion. In the preceding example, the payload value is not optional, the structure is defined by a schema named server1.DEBEZIUM.CUSTOMERS.Key, and there is one required field named id of type int32. The value of the key’s payload field indicates 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 inventory.customers table (output from the connector named server1) whose id primary key column had a value of 1004.

Change event values

The structure of a value in a change event message mirrors the structure of the message key in the change event in the message, and contains both a schema section and a payload section.

Payload of a change event value

An envelope structure in the payload sections of a change event value contains the following fields:

op

A mandatory field that contains a string value describing the type of operation. The op field in the payload of an Oracle connector change event value contains one of the following values: c (create or insert), u (update), d (delete), or r (read, which indicates a snapshot).

before

An optional field that, if present, describes the state of the row before the event occurred. The structure is described by the server1.INVENTORY.CUSTOMERS.Value Kafka Connect schema, which the server1 connector uses for all rows in the inventory.customers table.

after

An optional field that, if present, contains the state of a row after a change occurs. The structure is described by the same server1.INVENTORY.CUSTOMERS.Value Kafka Connect schema that is used for the before field.

source

A mandatory field that contains a structure that describes the source metadata for the event. In the case of the Oracle connector, the structure includes the following fields:

  • The Debezium version.

  • The connector name.

  • Whether the event is part of an ongoing snapshot or not.

  • The transaction id (not includes for snapshots).

  • The SCN of the change.

  • A timestamp that indicates when the record in the source database changed (for snapshots, the timestamp indicates when the snapshot occurred).

    The commit_scn field is optional and describes the SCN of the transaction commit that the change event participates within. This field is only present when using the LogMiner connection adapter.

ts_ms

An optional field that, if present, contains the time (based on the system clock in the JVM that runs the Kafka Connect task) at which the connector processed the event.

Schema of a change event value

The schema portion of the event message’s value contains a schema that describes the envelope structure of the payload and the nested fields within it.

create events

The following example shows the value of a create event value from the customers table that is described in the change event keys example:

{
    "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": "server1.DEBEZIUM.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": "server1.DEBEZIUM.CUSTOMERS.Value",
                "field": "after"
            },
            {
                "type": "struct",
                "fields": [
                    {
                        "type": "string",
                        "optional": true,
                        "field": "version"
                    },
                    {
                        "type": "string",
                        "optional": false,
                        "field": "name"
                    },
                    {
                        "type": "int64",
                        "optional": true,
                        "field": "ts_ms"
                    },
                    {
                        "type": "string",
                        "optional": true,
                        "field": "txId"
                    },
                    {
                        "type": "string",
                        "optional": true,
                        "field": "scn"
                    },
                    {
                        "type": "string",
                        "optional": true,
                        "field": "commit_scn"
                    },
                    {
                        "type": "boolean",
                        "optional": true,
                        "field": "snapshot"
                    }
                ],
                "optional": false,
                "name": "io.debezium.connector.oracle.Source",
                "field": "source"
            },
            {
                "type": "string",
                "optional": false,
                "field": "op"
            },
            {
                "type": "int64",
                "optional": true,
                "field": "ts_ms"
            }
        ],
        "optional": false,
        "name": "server1.DEBEZIUM.CUSTOMERS.Envelope"
    },
    "payload": {
        "before": null,
        "after": {
            "ID": 1004,
            "FIRST_NAME": "Anne",
            "LAST_NAME": "Kretchmar",
            "EMAIL": "annek@noanswer.org"
        },
        "source": {
            "version": "2.0.0.Alpha1",
            "name": "server1",
            "ts_ms": 1520085154000,
            "txId": "6.28.807",
            "scn": "2122185",
            "commit_scn": "2122185",
            "snapshot": false
        },
        "op": "c",
        "ts_ms": 1532592105975
    }
}

In the preceding example, notice how the event defines the following schema:

  • The envelope (server1.DEBEZIUM.CUSTOMERS.Envelope).

  • The source structure (io.debezium.connector.oracle.Source, which is specific to the Oracle connector and reused across all events).

  • 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 the schemas for all other tables. As a result, when you use the Avro converter, the Avro schemas for tables in each logical source have their own evolution and history.

The payload portion of this event’s value, provides information about the event. It describes that a row was created (op=c), and shows that the after field value contains the values that were inserted into the ID, FIRST_NAME, LAST_NAME, and EMAIL columns of the row.

By default, the JSON representations of events are much larger than the rows that they describe. The larger size is due to the JSON representation including both the schema and payload portions of a message. You can use the Avro Converter to decrease the size of messages that the connector writes to Kafka topics.

update events

The following example shows an update change event that the connector captures from the same table as the preceding create event.

{
    "schema": { ... },
    "payload": {
        "before": {
            "ID": 1004,
            "FIRST_NAME": "Anne",
            "LAST_NAME": "Kretchmar",
            "EMAIL": "annek@noanswer.org"
        },
        "after": {
            "ID": 1004,
            "FIRST_NAME": "Anne",
            "LAST_NAME": "Kretchmar",
            "EMAIL": "anne@example.com"
        },
        "source": {
            "version": "2.0.0.Alpha1",
            "name": "server1",
            "ts_ms": 1520085811000,
            "txId": "6.9.809",
            "scn": "2125544",
            "commit_scn": "2125544",
            "snapshot": false
        },
        "op": "u",
        "ts_ms": 1532592713485
    }
}

The payload has the same structure as the payload of a create (insert) event, but the following values are different:

  • The value of the op field is u, signifying that this row changed because of an update.

  • The before field shows the former state of the row with the values that were present before the update database commit.

  • The after field shows the updated state of the row, with the EMAIL value now set to anne@example.com.

  • The structure of the source field includes the same fields as before, but the values are different, because the connector captured the event from a different position in the redo log.

  • The ts_ms field shows the timestamp that indicates when Debezium processed the event.

The payload section reveals several other useful pieces of information. For example, by comparing the before and after structures, we can determine how a row changed as the result of a commit. The source structure provides information about Oracle’s record of this change, providing traceability. It also gives us insight into when this event occurred in relation to other events in this topic and in other topics. Did it occur before, after, or as part of the same commit as another event?

When the columns for a row’s primary/unique key are updated, the value of the row’s key changes. As a result, Debezium emits three events after such an update:

  • A DELETE event.

  • A tombstone event with the old key for the row.

  • An INSERT event that provides the new key for the row.

delete events

The following example shows a delete event for the table that is shown in the preceding create and update event examples. The schema portion of the delete event is identical to the schema portion for those events.

{
    "schema": { ... },
    "payload": {
        "before": {
            "ID": 1004,
            "FIRST_NAME": "Anne",
            "LAST_NAME": "Kretchmar",
            "EMAIL": "anne@example.com"
        },
        "after": null,
        "source": {
            "version": "2.0.0.Alpha1",
            "name": "server1",
            "ts_ms": 1520085153000,
            "txId": "6.28.807",
            "scn": "2122184",
            "commit_scn": "2122184",
            "snapshot": false
        },
        "op": "d",
        "ts_ms": 1532592105960
    }
}

The payload portion of the event reveals several differences when compared to the payload of a create or update event:

  • The value of the op field is d, signifying that the row was deleted.

  • The before field shows the former state of the row that was deleted with the database commit.

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

  • The structure of the source field includes many of the keys that exist in create or update events, but the values in the ts_ms, scn, and txId fields are different.

  • The ts_ms shows a timestamp that indicates when Debezium processed this event.

The delete event provides consumers with the information that they require to process the removal of this row.

The Oracle 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 shown in the preceding example still works with log compaction, because Kafka is able to remove all earlier messages that use the same key. The message value must be set to null to instruct Kafka to remove all messages that share the same key. To make this possible, by default, Debezium’s Oracle connector always follows a delete event with a special tombstone event that has the same key but null value. You can change the default behavior by setting the connector property tombstones.on.delete.

truncate events

A truncate change event signals that a table has been truncated. The message key is null in this case, the message value looks like this:

{
    "schema": { ... },
    "payload": {
        "before": null,
        "after": null,
        "source": { (1)
            "version": "2.0.0.Alpha1",
            "connector": "oracle",
            "name": "oracle_server",
            "ts_ms": 1638974535000,
            "snapshot": "false",
            "db": "ORCLPDB1",
            "sequence": null,
            "schema": "DEBEZIUM",
            "table": "TEST_TABLE",
            "txId": "02000a0037030000",
            "scn": "13234397",
            "commit_scn": "13271102",
            "lcr_position": null
        },
        "op": "t", (2)
        "ts_ms": 1638974558961, (3)
        "transaction": null
    }
}
Table 5. Descriptions of truncate event value fields
Item Field name Description

1

source

Mandatory field that describes the source metadata for the event. In a truncate event value, the source field structure is the same as for create, update, and delete events for the same table, provides this metadata:

  • Debezium version

  • Connector type and name

  • Database and table that contains the new row

  • Schema name

  • If the event was part of a snapshot (always false for truncate events)

  • ID of the transaction in which the operation was performed

  • SCN of the operation

  • Timestamp for when the change was made in the database

2

op

Mandatory string that describes the type of operation. The op field value is t, signifying that this table was truncated.

3

ts_ms

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

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

Because truncate events represent changes made to an entire table, and have no message key, in topics with multiple partitions, there is no guarantee that consumers receive truncate events and change events (create, update, etc.) for to a table in order. For example, when a consumer reads events from different partitions, it might receive an update event for a table after it receives a truncate event for the same table. Ordering can be guaranteed only if a topic uses a single partition.

If you do not want to capture truncate events, use the skipped.operations option to filter them out.

Data type mappings

When the Debezium Oracle connector detects a change in the value of a table row, it emits a change event that represents the change. Each change event record is structured in the same way as the original table, with the event record containing a field for each column value. The data type of a table column determines how the connector represents the column’s values in change event fields, as shown in the tables in the following sections.

For each column in a table, Debezium maps the source data type to a literal type and, and in some cases, a semantic type, in the corresponding event field.

Literal types

Describe how the value is literally represented, using one of the following Kafka Connect schema types: INT8, INT16, INT32, INT64, FLOAT32, FLOAT64, BOOLEAN, STRING, BYTES, ARRAY, MAP, and STRUCT.

Semantic types

Describe how the Kafka Connect schema captures the meaning of the field, by using the name of the Kafka Connect schema for the field.

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

For some Oracle large object (CLOB, NCLOB, and BLOB) and numeric data types, you can manipulate the way that the connector performs the type mapping by changing default configuration property settings. For more information about how Debezium properties control mappings for these data types, see Binary and Character LOB types and Numeric types.

Support for further data types is planned for subsequent releases. Please file a JIRA issue for any specific types that might be missing.

Character types

The following table describes how the connector maps basic character types.

Table 6. Mappings for Oracle basic character types
Oracle Data Type Literal type (schema type) Semantic type (schema name) and Notes

CHAR[(M)]

STRING

n/a

NCHAR[(M)]

STRING

n/a

NVARCHAR2[(M)]

STRING

n/a

VARCHAR[(M)]

STRING

n/a

VARCHAR2[(M)]

STRING

n/a

Binary and Character LOB types

Support for BLOB, CLOB, and NCLOB is currently in incubating state, that is, the exact semantics, configuration options and so forth might change in future revisions, based on feedback we receive. Please let us know if you encounter any problems while using these data types.

The following table describes how the connector maps binary and character large object (LOB) data types.

Table 7. Mappings for Oracle binary and character LOB types
Oracle Data Type Literal type (schema type) Semantic type (schema name) and Notes

BFILE

n/a

This data type is not supported

BLOB

BYTES

Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the binary.handling.mode connector configuration property setting.

CLOB

STRING

n/a

LONG

n/a

This data type is not supported.

LONG RAW

n/a

This data type is not supported.

NCLOB

STRING

n/a

RAW

n/a

This data type is not supported.

Oracle only supplies column values for CLOB, NCLOB, and BLOB data types if they’re explicitly set or changed in a SQL statement. As a result, change events never contain the value of an unchanged CLOB, NCLOB, or BLOB column. Instead, they contain placeholders as defined by the connector property, unavailable.value.placeholder.

If the value of a CLOB, NCLOB, or BLOB column is updated, the new value is placed in the after element of the corresponding update change event. The before element contains the unavailable value placeholder.

Numeric types

The following table describes how the Debezium Oracle connector maps numeric types.

You can modify the way that the connector maps the Oracle DECIMAL, NUMBER, NUMERIC, and REAL data types by changing the value of the connector’s decimal.handling.mode configuration property. When the property is set to its default value of precise, the connector maps these Oracle data types to the Kafka Connect org.apache.kafka.connect.data.Decimal logical type, as indicated in the table. When the value of the property is set to double or string, the connector uses alternate mappings for some Oracle data types. For more information, see the Semantic type and Notes column in the following table.

Table 8. Mappings for Oracle numeric data types
Oracle Data Type Literal type (schema type) Semantic type (schema name) and Notes

BINARY_FLOAT

FLOAT32

n/a

BINARY_DOUBLE

FLOAT64

n/a

DECIMAL[(P, S)]

BYTES / INT8 / INT16 / INT32 / INT64

org.apache.kafka.connect.data.Decimal if using BYTES

Handled equivalently to NUMBER (note that S defaults to 0 for DECIMAL).

When the decimal.handling.mode property is set to double, the connector represents DECIMAL values as Java double values with schema type FLOAT64.

When the decimal.handling.mode property is set to string, the connector represents DECIMAL values as their formatted string representation with schema type STRING.

DOUBLE PRECISION

STRUCT

io.debezium.data.VariableScaleDecimal

Contains a structure with two fields: scale of type INT32 that contains the scale of the transferred value and value of type BYTES containing the original value in an unscaled form.

FLOAT[(P)]

STRUCT

io.debezium.data.VariableScaleDecimal

Contains a structure with two fields: scale of type INT32 that contains the scale of the transferred value and value of type BYTES containing the original value in an unscaled form.

INTEGER, INT

BYTES

org.apache.kafka.connect.data.Decimal

INTEGER is mapped in Oracle to NUMBER(38,0) and hence can hold values larger than any of the INT types could store

NUMBER[(P[, *])]

STRUCT

io.debezium.data.VariableScaleDecimal

Contains a structure with two fields: scale of type INT32 that contains the scale of the transferred value and value of type BYTES containing the original value in an unscaled form.

When the decimal.handling.mode property is set to double, the connector represents NUMBER values as Java double values with schema type FLOAT64.

When the decimal.handling.mode property is set to string, the connector represents NUMBER values as their formatted string representation with schema type STRING.

NUMBER(P, S <= 0)

INT8 / INT16 / INT32 / INT64

NUMBER columns with a scale of 0 represent integer numbers. A negative scale indicates rounding in Oracle, for example, a scale of -2 causes rounding to hundreds.

Depending on the precision and scale, one of the following matching Kafka Connect integer type is chosen:

  • P - S < 3, INT8

  • P - S < 5, INT16

  • P - S < 10, INT32

  • P - S < 19, INT64

  • P - S >= 19, BYTES (org.apache.kafka.connect.data.Decimal)

When the decimal.handling.mode property is set to double, the connector represents NUMBER values as Java double values with schema type FLOAT64.

When the decimal.handling.mode property is set to string, the connector represents NUMBER values as their formatted string representation with schema type STRING.

NUMBER(P, S > 0)

BYTES

org.apache.kafka.connect.data.Decimal

NUMERIC[(P, S)]

BYTES / INT8 / INT16 / INT32 / INT64

org.apache.kafka.connect.data.Decimal if using BYTES

Handled equivalently to NUMBER (note that S defaults to 0 for NUMERIC).

When the decimal.handling.mode property is set to double, the connector represents NUMERIC values as Java double values with schema type FLOAT64.

When the decimal.handling.mode property is set to string, the connector represents NUMERIC values as their formatted string representation with schema type STRING.

SMALLINT

BYTES

org.apache.kafka.connect.data.Decimal

SMALLINT is mapped in Oracle to NUMBER(38,0) and hence can hold values larger than any of the INT types could store

REAL

STRUCT

io.debezium.data.VariableScaleDecimal

Contains a structure with two fields: scale of type INT32 that contains the scale of the transferred value and value of type BYTES containing the original value in an unscaled form.

When the decimal.handling.mode property is set to double, the connector represents REAL values as Java double values with schema type FLOAT64.

When the decimal.handling.mode property is set to string, the connector represents REAL values as their formatted string representation with schema type STRING.

Boolean types

Oracle does not provide native support for a BOOLEAN data type. However, it is common practice to use other data types with certain semantics to simulate the concept of a logical BOOLEAN data type.

To enable you to convert source columns to Boolean data types, Debezium provides a NumberOneToBooleanConverter custom converter that you can use in one of the following ways:

  • Map all NUMBER(1) columns to a BOOLEAN type.

  • Enumerate a subset of columns by using a comma-separated list of regular expressions.
    To use this type of conversion, you must set the converters configuration property with the selector parameter, as shown in the following example:

    converters=boolean
    boolean.type=io.debezium.connector.oracle.converters.NumberOneToBooleanConverter
    boolean.selector=.*MYTABLE.FLAG,.*.IS_ARCHIVED

Temporal types

Other than the Oracle INTERVAL, TIMESTAMP WITH TIME ZONE, and TIMESTAMP WITH LOCAL TIME ZONE data types, the way that the connector converts temporal types depends 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 determines the literal 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:

Oracle data type Literal type (schema type) Semantic type (schema name) and Notes

DATE

INT64

io.debezium.time.Timestamp

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

INTERVAL DAY[(M)] TO SECOND

FLOAT64

io.debezium.time.MicroDuration

The number of micro seconds for a time interval using the 365.25 / 12.0 formula for days per month average.

io.debezium.time.Interval (when interval.handling.mode is set to string)

The string representation of the interval value that follows the pattern P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S, for example, P1Y2M3DT4H5M6.78S.

INTERVAL YEAR[(M)] TO MONTH

FLOAT64

io.debezium.time.MicroDuration

The number of micro seconds for a time interval using the 365.25 / 12.0 formula for days per month average.

io.debezium.time.Interval (when interval.handling.mode is set to string)

The string representation of the interval value that follows the pattern P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S, for example, P1Y2M3DT4H5M6.78S.

TIMESTAMP(0 - 3)

INT64

io.debezium.time.Timestamp

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

TIMESTAMP, TIMESTAMP(4 - 6)

INT64

io.debezium.time.MicroTimestamp

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

TIMESTAMP(7 - 9)

INT64

io.debezium.time.NanoTimestamp

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

TIMESTAMP WITH TIME ZONE

STRING

io.debezium.time.ZonedTimestamp

A string representation of a timestamp with timezone information.

TIMESTAMP WITH LOCAL TIME ZONE

STRING

io.debezium.time.ZonedTimestamp

A string representation of a timestamp in UTC.

When the time.precision.mode configuration property is set to connect, then the connector uses the predefined Kafka Connect logical types. This can be useful when consumers only know about the built-in Kafka Connect logical types and are unable to handle variable-precision time values. Because the level of precision that Oracle supports exceeds the level that the logical types in Kafka Connect support, if you set time.precision.mode to connect, a loss of precision results when the fractional second precision value of a database column is greater than 3:

Oracle data type Literal type (schema type) Semantic type (schema name) and Notes

DATE

INT32

org.apache.kafka.connect.data.Date

Represents the number of days since the UNIX epoch.

INTERVAL DAY[(M)] TO SECOND

FLOAT64

io.debezium.time.MicroDuration

The number of micro seconds for a time interval using the 365.25 / 12.0 formula for days per month average.

io.debezium.time.Interval (when interval.handling.mode is set to string)

The string representation of the interval value that follows the pattern P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S, for example, P1Y2M3DT4H5M6.78S.

INTERVAL YEAR[(M)] TO MONTH

FLOAT64

io.debezium.time.MicroDuration

The number of micro seconds for a time interval using the 365.25 / 12.0 formula for days per month average.

io.debezium.time.Interval (when interval.handling.mode is set to string)

The string representation of the interval value that follows the pattern P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S, for example, P1Y2M3DT4H5M6.78S.

TIMESTAMP(0 - 3)

INT64

org.apache.kafka.connect.data.Timestamp

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

TIMESTAMP(4 - 6)

INT64

org.apache.kafka.connect.data.Timestamp

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

TIMESTAMP(7 - 9)

INT64

org.apache.kafka.connect.data.Timestamp

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

TIMESTAMP WITH TIME ZONE

STRING

io.debezium.time.ZonedTimestamp

A string representation of a timestamp with timezone information.

TIMESTAMP WITH LOCAL TIME ZONE

STRING

io.debezium.time.ZonedTimestamp

A string representation of a timestamp in UTC.

ROWID types

The following table describes how the connector maps ROWID (row address) data types.

Table 9. Mappings for Oracle ROWID data types
Oracle Data Type Literal type (schema type) Semantic type (schema name) and Notes

ROWID

STRING

This data type is not supported when using Oracle XStream.

UROWID

n/a

This data type is not supported.

User-defined types

Oracle enables you to define custom data types to provide flexibility when the built-in data types do not satisfy your requirements. There are a several user-defined types such as Object types, REF data types, Varrays, and Nested Tables. At this time, you cannot use the Debezium Oracle connector with any of these user-defined types.

Oracle-supplied types

Oracle provides SQL-based interfaces that you can use to define new types when the built-in or ANSI-supported types are insufficient. Oracle offers several commonly used data types to serve a broad array of purposes such as Any, XML, or Spatial types. At this time, you cannot use the Debezium Oracle connector with any of these data types.

Default Values

If a default value is specified for a column in the database schema, the Oracle connector will attempt to propagate this value to the schema of the corresponding Kafka record field. Most common data types are supported, including:

  • Character types (CHAR, NCHAR, VARCHAR, VARCHAR2, NVARCHAR, NVARCHAR2)

  • Numeric types (INTEGER, NUMERIC, etc.)

  • Temporal types (DATE, TIMESTAMP, INTERVAL, etc.)

If a temporal type uses a function call such as TO_TIMESTAMP or TO_DATE to represent the default value, the connector will resolve the default value by making an additional database call to evaluate the function. For example, if a DATE column is defined with the default value of TO_DATE('2021-01-02', 'YYYY-MM-DD'), the column’s default value will be the number of days since the UNIX epoch for that date or 18629 in this case.

If a temporal type uses the SYSDATE constant to represent the default value, the connector will resolve this based on whether the column is defined as NOT NULL or NULL. If the column is nullable, no default value will be set; however, if the column isn’t nullable then the default value will be resolved as either 0 (for DATE or TIMESTAMP(n) data types) or 1970-01-01T00:00:00Z (for TIMESTAMP WITH TIME ZONE or TIMESTAMP WITH LOCAL TIME ZONE data types). The default value type will be numeric except if the column is a TIMESTAMP WITH TIME ZONE or TIMESTAMP WITH LOCAL TIME ZONE in which case its emitted as a string.

Setting up Oracle

The following steps are necessary to set up Oracle for use with the Debezium Oracle connector. These steps assume the use of the multi-tenancy configuration with a container database and at least one pluggable database. If you do not intend to use a multi-tenant configuration, it might be necessary to adjust the following steps.

For information about using Vagrant to set up Oracle in a virtual machine, see the Debezium Vagrant Box for Oracle database GitHub repository.

Compatibility with Oracle installation types

An Oracle database can be installed either as a standalone instance or using Oracle Real Application Cluster (RAC). The Debezium Oracle connector is compatible with both types of installation.

Schemas excluded from capture

When the Debezium Oracle connector captures tables, it automatically excludes tables from the following schemas:

  • appqossys

  • audsys

  • ctxsys

  • dvsys

  • dbsfwuser

  • dbsnmp

  • qsmadmin_internal

  • lbacsys

  • mdsys

  • ojvmsys

  • olapsys

  • orddata

  • ordsys

  • outln

  • sys

  • system

  • wmsys

  • xdb

To enable the connector to capture changes from a table, the table must use a schema that is not named in the preceding list.

Preparing the database

Configuration needed for Oracle LogMiner
ORACLE_SID=ORACLCDB dbz_oracle sqlplus /nolog

CONNECT sys/top_secret AS SYSDBA
alter system set db_recovery_file_dest_size = 10G;
alter system set db_recovery_file_dest = '/opt/oracle/oradata/recovery_area' scope=spfile;
shutdown immediate
startup mount
alter database archivelog;
alter database open;
-- Should now "Database log mode: Archive Mode"
archive log list

exit;

To enable Debezium to capture the before state of changed database rows, you must also enable supplemental logging for captured tables or for the entire database. The following example illustrates how to configure supplemental logging for all columns in a single inventory.customers table.

ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS;

Enabling supplemental logging for all table columns increases the volume of the Oracle redo logs. To prevent excessive growth in the size of the logs, apply the preceding configuration selectively.

Minimal supplemental logging must be enabled at the database level and can be configured as follows.

ALTER DATABASE ADD SUPPLEMENTAL LOG DATA;

Redo log sizing

Depending on the database configuration, the size and number of redo logs might not be sufficient to achieve acceptable performance. Before you set up the Debezium Oracle connector, ensure that the capacity of the redo logs is sufficient to support the database.

The capacity of the redo logs for a database must be sufficient to store its data dictionary. In general, the size of the data dictionary increases with the number of tables and columns in the database. If the redo log lacks sufficient capacity, both the database and the Debezium connector might experience performance problems.

Consult with your database administrator to evaluate whether the database might require increased log capacity.

Creating users for the connector

For the Debezium Oracle connector to capture change events, it must run as an Oracle LogMiner user that has specific permissions. The following example shows the SQL for creating an Oracle user account for the connector in a multi-tenant database model.

The connector captures database changes that are made by its own Oracle user account. However, it does not capture changes that are made by the SYS or SYSTEM user accounts.

Creating the connector’s LogMiner user
sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba
  CREATE TABLESPACE logminer_tbs DATAFILE '/opt/oracle/oradata/ORCLCDB/logminer_tbs.dbf'
    SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
  exit;

sqlplus sys/top_secret@//localhost:1521/ORCLPDB1 as sysdba
  CREATE TABLESPACE logminer_tbs DATAFILE '/opt/oracle/oradata/ORCLCDB/ORCLPDB1/logminer_tbs.dbf'
    SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
  exit;

sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba

  CREATE USER c##dbzuser IDENTIFIED BY dbz
    DEFAULT TABLESPACE logminer_tbs
    QUOTA UNLIMITED ON logminer_tbs
    CONTAINER=ALL;

  GRANT CREATE SESSION TO c##dbzuser CONTAINER=ALL; (1)
  GRANT SET CONTAINER TO c##dbzuser CONTAINER=ALL; (2)
  GRANT SELECT ON V_$DATABASE to c##dbzuser CONTAINER=ALL; (3)
  GRANT FLASHBACK ANY TABLE TO c##dbzuser CONTAINER=ALL; (4)
  GRANT SELECT ANY TABLE TO c##dbzuser CONTAINER=ALL; (5)
  GRANT SELECT_CATALOG_ROLE TO c##dbzuser CONTAINER=ALL; (6)
  GRANT EXECUTE_CATALOG_ROLE TO c##dbzuser CONTAINER=ALL; (7)
  GRANT SELECT ANY TRANSACTION TO c##dbzuser CONTAINER=ALL; (8)
  GRANT LOGMINING TO c##dbzuser CONTAINER=ALL; (9)

  GRANT CREATE TABLE TO c##dbzuser CONTAINER=ALL; (10)
  GRANT LOCK ANY TABLE TO c##dbzuser CONTAINER=ALL; (11)
  GRANT CREATE SEQUENCE TO c##dbzuser CONTAINER=ALL; (12)

  GRANT EXECUTE ON DBMS_LOGMNR TO c##dbzuser CONTAINER=ALL; (13)
  GRANT EXECUTE ON DBMS_LOGMNR_D TO c##dbzuser CONTAINER=ALL; (14)

  GRANT SELECT ON V_$LOG TO c##dbzuser CONTAINER=ALL; (15)
  GRANT SELECT ON V_$LOG_HISTORY TO c##dbzuser CONTAINER=ALL; (16)
  GRANT SELECT ON V_$LOGMNR_LOGS TO c##dbzuser CONTAINER=ALL; (17)
  GRANT SELECT ON V_$LOGMNR_CONTENTS TO c##dbzuser CONTAINER=ALL; (18)
  GRANT SELECT ON V_$LOGMNR_PARAMETERS TO c##dbzuser CONTAINER=ALL; (19)
  GRANT SELECT ON V_$LOGFILE TO c##dbzuser CONTAINER=ALL; (20)
  GRANT SELECT ON V_$ARCHIVED_LOG TO c##dbzuser CONTAINER=ALL; (21)
  GRANT SELECT ON V_$ARCHIVE_DEST_STATUS TO c##dbzuser CONTAINER=ALL; (22)
  GRANT SELECT ON V_$TRANSACTION TO c##dbzuser CONTAINER=ALL; (23)

  exit;
Table 10. Descriptions of permissions / grants
Item Role name Description

1

CREATE SESSION

Enables the connector to connect to Oracle.

2

SET CONTAINER

Enables the connector to switch between pluggable databases. This is only required when the Oracle installation has container database support (CDB) enabled.

3

SELECT ON V_$DATABASE

Enables the connector to read the V$DATABASE table.

4

FLASHBACK ANY TABLE

Enables the connector to perform Flashback queries, which is how the connector performs the initial snapshot of data.

5

SELECT ANY TABLE

Enables the connector to read any table.

6

SELECT_CATALOG_ROLE

Enables the connector to read the data dictionary, which is needed by Oracle LogMiner sessions.

7

EXECUTE_CATALOG_ROLE

Enables the connector to write the data dictionary into the Oracle redo logs, which is needed to track schema changes.

8

SELECT ANY TRANSACTION

Enables the snapshot process to perform a Flashback snapshot query against any transaction. When FLASHBACK ANY TABLE is granted, this should also be granted.

9

LOGMINING

This role was added in newer versions of Oracle as a way to grant full access to Oracle LogMiner and its packages. On older versions of Oracle that don’t have this role, you can ignore this grant.

10

CREATE TABLE

Enables the connector to create its flush table in its default tablespace. The flush table allows the connector to explicitly control flushing of the LGWR internal buffers to disk.

11

LOCK ANY TABLE

Enables the connector to lock tables during schema snapshot. If snapshot locks are explicitly disabled via configuration, this grant can be safely ignored.

12

CREATE ANY SEQUENCE

Enables the connector to create a sequence in its default tablespace.

13

EXECUTE ON DBMS_LOGMNR

Enables the connector to run methods in the DBMS_LOGMNR package. This is required to interact with Oracle LogMiner. On newer versions of Oracle this is granted via the LOGMINING role but on older versions, this must be explicitly granted.

14

EXECUTE ON DBMS_LOGMNR_D

Enables the connector to run methods in the DBMS_LOGMNR_D package. This is required to interact with Oracle LogMiner. On newer versions of Oracle this is granted via the LOGMINING role but on older versions, this must be explicitly granted.

15 to 23

SELECT ON V_$…​.

Enables the connector to read these tables. The connector must be able to read information about the Oracle redo and archive logs, and the current transaction state, to prepare the Oracle LogMiner session. Without these grants, the connector cannot operate.

Standby databases

An Oracle database can be configured with either a physical or a logical standby environment to provide for recovery after of a production failure. At this time, the Debezium Oracle connector cannot use a physical or logical standby database as the change event source. There is an open Jira issue to investigate this support.

Failover databases

It is customary for a logical or physical standby to exist in the case of an Oracle production failure. When a failure occurs and the standby instance is promoted to production, the database must be opened for read/write transactions before the Debezium Oracle connector can connect to the database.

In the case of a physical standby, the standby is an exact copy of production, which implies that the SCN values are identical. When using a physical standby, it is sufficient to reconfigure the Debezium Oracle connector to use the hostname of the standby once the database is open.

In the case of a logical standby, the standby is not an exact copy of the production database, so the SCN offsets in the standby differ from those in the production database. If you use a logical standby, to help ensure that Debezium does not miss any change events, after the database is open, configure a new connector and perform a new database snapshot.

Deployment

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

Prerequisites
Procedure
  1. Download the Debezium Oracle connector plug-in archive.

  2. Extract the files into your Kafka Connect environment.

  3. Download the JDBC driver for Oracle from Maven Central and extract the downloaded driver file to the directory that contains the Debezium Oracle connector JAR file.

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

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

Debezium Oracle connector configuration

Typically, you register a Debezium Oracle connector by submitting a JSON request that specifies the configuration properties for the connector. The following example shows a JSON request for registering an instance of the Debezium Oracle connector with logical name server1 at port 1521:

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

Example: Debezium Oracle connector configuration
{
    "name": "inventory-connector",  (1)
    "config": {
        "connector.class" : "io.debezium.connector.oracle.OracleConnector",  (2)
        "database.hostname" : "<ORACLE_IP_ADDRESS>",  (3)
        "database.port" : "1521",  (4)
        "database.user" : "c##dbzuser",  (5)
        "database.password" : "dbz",   (6)
        "database.dbname" : "ORCLCDB",  (7)
        "database.server.name" : "server1",  (8)
        "tasks.max" : "1",  (9)
        "database.pdb.name" : "ORCLPDB1",  (10)
        "database.history.kafka.bootstrap.servers" : "kafka:9092", (11)
        "database.history.kafka.topic": "schema-changes.inventory"  (12)
    }
}
1 The name that is assigned to the connector when you register it with a Kafka Connect service.
2 The name of this Oracle connector class.
3 The address of the Oracle instance.
4 The port number of the Oracle instance.
5 The name of the Oracle user, as specified in Creating users for the connector.
6 The password for the Oracle user, as specified in Creating users for the connector.
7 The name of the database to capture changes from.
8 Logical name that identifies and provides a namespace for the Oracle database server from which the connector captures changes.
9 The maximum number of tasks to create for this connector.
10 The name of the Oracle pluggable database that the connector captures changes from. Used in container database (CDB) installations only.
11 The list of Kafka brokers that this connector uses to write and recover DDL statements to the database history topic.
12 The name of the database history topic where the connector writes and recovers DDL statements. This topic is for internal use only and should not be used by consumers.

In the previous example, the database.hostname and database.port properties are used to define the connection to the database host. However, in more complex Oracle deployments, or in deployments that use TNS names, you can use an alternative method in which you specify a JDBC URL.

The following JSON example shows the same configuration as in the preceding example, except that it uses a JDBC URL to connect to the database.

Example: Debezium Oracle connector configuration that uses a JDBC URL to connect to the database
{
    "name": "inventory-connector",
    "config": {
        "connector.class" : "io.debezium.connector.oracle.OracleConnector",
        "tasks.max" : "1",
        "database.server.name" : "server1",
        "database.user" : "c##dbzuser",
        "database.password" : "dbz",
        "database.url": "jdbc:oracle:thin:@(DESCRIPTION=(ADDRESS_LIST=(LOAD_BALANCE=OFF)(FAILOVER=ON)(ADDRESS=(PROTOCOL=TCP)(HOST=<oracle ip 1>)(PORT=1521))(ADDRESS=(PROTOCOL=TCP)(HOST=<oracle ip 2>)(PORT=1521)))(CONNECT_DATA=SERVICE_NAME=)(SERVER=DEDICATED)))",
        "database.dbname" : "ORCLCDB",
        "database.pdb.name" : "ORCLPDB1",
        "database.history.kafka.bootstrap.servers" : "kafka:9092",
        "database.history.kafka.topic": "schema-changes.inventory"
    }
}

Pluggable vs Non-Pluggable databases

Oracle Database supports the following deployment types:

Container database (CDB)

A database that can contain multiple pluggable databases (PDBs). Database clients connect to each PDB as if it were a standard, non-CDB database.

Non-container database (non-CDB)

A standard Oracle database, which does not support the creation of pluggable databases.

Example: Debezium connector configuration for CDB deployments
{
  "config": {
    "connector.class" : "io.debezium.connector.oracle.OracleConnector",
    "tasks.max" : "1",
    "database.server.name" : "server1",
    "database.hostname" : "<oracle ip>",
    "database.port" : "1521",
    "database.user" : "c##dbzuser",
    "database.password" : "dbz",
    "database.dbname" : "ORCLCDB",
    "database.pdb.name" : "ORCLPDB1",
    "database.history.kafka.bootstrap.servers" : "kafka:9092",
    "database.history.kafka.topic": "schema-changes.inventory"
  }
}

When you configure a Debezium Oracle connector for use with an Oracle CDB, you must specify a value for the property database.pdb.name, which names the PDB that you want the connector to capture changes from. For non-CDB installation, do not specify the database.pdb.name property.

Example: Debezium Oracle connector configuration for non-CDB deployments
{
    "config": {
        "connector.class" : "io.debezium.connector.oracle.OracleConnector",
        "tasks.max" : "1",
        "database.server.name" : "server1",
        "database.hostname" : "<oracle ip>",
        "database.port" : "1521",
        "database.user" : "c##dbzuser",
        "database.password" : "dbz",
        "database.dbname" : "ORCLCDB",
        "database.history.kafka.bootstrap.servers" : "kafka:9092",
        "database.history.kafka.topic": "schema-changes.inventory"
    }
}

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

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

  • Connects to the Oracle database.

  • Reads the redo log.

  • Records change events to Kafka topics.

Adding connector configuration

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

Prerequisites
Procedure
  1. Create a configuration for the Oracle connector.

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

Results

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

Connector properties

The Debezium Oracle connector has numerous configuration properties that you can use to achieve the right connector behavior for your application. Many properties have default values. Information about the properties is organized as follows:

Required Debezium Oracle connector configuration properties

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

Property

Default

Description

No default

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

No default

The name of the Java class for the connector. Always use a value of io.debezium.connector.oracle.OracleConnector for the Oracle connector.

No default

Enumerates a comma-separated list of the symbolic names of the custom converter instances that the connector can use.
For example, boolean.
This property is required to enable the connector to use a custom converter.

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

<converterSymbolicName>.type

For example,

boolean.type: io.debezium.connector.oracle.converters.NumberOneToBooleanConverter

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

For example, to define a selector parameter that specifies the subset of columns that the boolean converter processes, add the following property:

boolean.selector: .*MYTABLE.FLAG,.*.IS_ARCHIVED

1

The maximum number of tasks to create for this connector. The Oracle connector always uses a single task and therefore does not use this value, so the default is always acceptable.

No default

IP address or hostname of the Oracle database server.

No default

Integer port number of the Oracle database server.

No default

Name of the Oracle user account that the connector uses to connect to the Oracle database server.

No default

Password to use when connecting to the Oracle database server.

No default

Name of the database to connect to. Must be the CDB name when working with the CDB + PDB model.

No default

Specifies the raw database JDBC URL. Use this property to provide flexibility in defining that database connection. Valid values include raw TNS names and RAC connection strings.

No default

Name of the Oracle pluggable database to connect to. Use this property with container database (CDB) installations only.

No default

Logical name that identifies and provides a namespace for the Oracle database server from which the connector captures changes. The value that you set is used as a prefix for all Kafka topic names that the connector emits. Specify a logical name that is unique among all connectors in your Debezium environment. The following characters are valid: alphanumeric characters, hyphens, dots, and underscores.

+

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

logminer

The adapter implementation that the connector uses when it streams database changes. You can set the following values: logminer(default):: The connector uses the native Oracle LogMiner API. xstream:: The connector uses the Oracle XStreams API.

initial

Specifies the mode that the connector uses to take snapshots of a captured table. You can set the following values:

initial

The snapshot includes the structure and data of the captured tables. Specify this value to populate topics with a complete representation of the data from the captured tables.

initial_only

The snapshot includes the structure and data of the captured tables. The connector performs an initial snapshot and then stops, without processing any subsequent changes.

schema_only

The snapshot includes only the structure of captured tables. Specify this value if you want the connector to capture data only for changes that occur after the snapshot.

schema_only_recovery

This is a recovery setting for a connector that has already been capturing changes. When you restart the connector, this setting enables recovery of a corrupted or lost database history topic. You might set it periodically to "clean up" a database history topic that has been growing unexpectedly. Database history topics require infinite retention. Note this mode is only safe to be used when it is guaranteed that no schema changes happened since the point in time the connector was shut down before and the point in time the snapshot is taken.

After the snapshot is complete, the connector continues to read change events from the database’s redo logs except when snapshot.mode is configured as initial_only.

For more information, see the table of snapshot.mode options.

shared

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

shared

Enables concurrent access to the table, but prevents any session from acquiring an exclusive table lock. The connector acquires a ROW SHARE level lock while it captures table schema.

none

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

All tables specified in table.include.list

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

This property does not affect the behavior of incremental snapshots.

No default

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

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

"snapshot.select.statement.overrides": "inventory.products,customers.orders"

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

snapshot.select.statement.overrides.<schemaName>.<tableName>

For example, snapshot.select.statement.overrides.customers.orders

Example:

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

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

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

No default

An optional, comma-separated list of regular expressions that match names of schemas for which you want to capture changes. Any schema name not included in schema.include.list is excluded from having its changes captured. By default, all non-system schemas have their changes captured. Do not also set the schema.exclude.list property. In environments that use the LogMiner implementation, you must use POSIX regular expressions only.

false

Boolean value that specifies whether the connector should parse and publish table and column comments on metadata objects. Enabling this option will bring the implications on memory usage. The number and size of logical schema objects is what largely impacts how much memory is consumed by the Debezium connectors, and adding potentially large string data to each of them can potentially be quite expensive.

No default

An optional, comma-separated list of regular expressions that match names of schemas for which you do not want to capture changes. Any schema whose name is not included in schema.exclude.list has its changes captured, with the exception of system schemas. Do not also set the schema.include.list property. In environments that use the LogMiner implementation, you must use POSIX regular expressions only.

No default

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be monitored. Tables that are not included in the include list are excluded from monitoring. Each table identifier uses the following format:

<schema_name>.<table_name>

By default, the connector monitors every non-system table in each monitored database. Do not use this property in combination with table.exclude.list. If you use the LogMiner implementation, use only POSIX regular expressions with this property.

No default

An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be excluded from monitoring. The connector captures change events from any table that is not specified in the exclude list. Specify the identifier for each table using the following format:

<schemaName>.<tableName>.

Do not use this property in combination with table.include.list. If you use the LogMiner implementation, use only POSIX regular expressions with this property.

No default

An optional comma-separated list of regular expressions that match the fully-qualified names of columns that want to include in the change event message values. Fully-qualified names for columns use the following format: +
`<Schema_name>.<table_name>.<column_name>


The primary key column is always included in an event’s key, even if you do not use this property to explicitly include its value. If you include this property in the configuration, do not also set the column.exclude.list property.

No default

An optional comma-separated list of regular expressions that match the fully-qualified names of columns that you want to exclude from change event message values. Fully-qualified column names use the following format:

<schema_name>.<table_name>.<column_name>

The primary key column is always included in an event’s key, even if you use this property to explicitly exclude its value. If you include this property in the configuration, do not set the column.include.list property.

n/a

An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns. Fully-qualified names for columns are of the form <schemaName>.<tableName>.<columnName>.

In the resulting change event record, the values for the specified columns are replaced with pseudonyms.

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

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

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

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

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

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

bytes

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

avro

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

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

  • none does not apply any adjustment.

precise

Specifies how the connector should handle floating point values for NUMBER, DECIMAL and NUMERIC columns. You can set one of the following options:

precise (default)

Represents values precisely by using java.math.BigDecimal values represented in change events in a binary form.

double

Represents values by using double values. Using double values is easier, but can result in a loss of precision.

string

Encodes values as formatted strings. Using the string option is easier to consume, but results in a loss of semantic information about the real type. For more information, see Numeric types.

numeric

Specifies how the connector should handle values for interval columns:

numeric represents intervals using approximate number of microseconds.

string represents intervals exactly by using the string pattern representation P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S. For example: P1Y2M3DT4H5M6.78S.

fail

Specifies how the connector should react to exceptions during processing of events. You can set one of the following options:

fail

Propagates the exception (indicating the offset of the problematic event), causing the connector to stop.

warn

Causes the problematic event to be skipped. The offset of the problematic event is then logged.

skip

Causes the problematic event to be skipped.

2048

A positive integer value that specifies the maximum size of each batch of events to process during each iteration of this connector.

8192

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

0 (disabled)

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

1000 (1 second)

Positive integer value that specifies the number of milliseconds the connector should wait during each iteration for new change events to appear.

true

Controls whether a delete event is followed by a tombstone event. The following values are possible:

true

For each delete operation, the connector emits a delete event and a subsequent tombstone event.

false

For each delete operation, the connector emits only a delete event.

After a source record is deleted, a tombstone event (the default behavior) enables Kafka to completely delete all events that share the key of the deleted row in topics that have log compaction enabled.

No default

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

By default, Debezium uses the primary key column of a table as the message key for records that it emits. In place of the default, or to specify a key for tables that lack a primary key, you can configure custom message keys based on one or more columns.
To establish a custom message key for a table, list the table, followed by the columns to use as the message key. Each list entry takes the following format:

<fullyQualifiedTableName>:<keyColumn>,<keyColumn>

To base a table key on multiple column names, insert commas between the column names.
Each fully-qualified table name is a regular expression in the following format:

<schemaName>.<tableName>

The property can include entries for multiple tables. Use a semicolon to separate table entries in the list.
The following example sets the message key for the tables inventory.customers and purchase.orders:

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

For the table inventory.customer, the columns pk1 and pk2 are specified as the message key. For the purchaseorders tables in any schema, the columns pk3 and pk4 server as the message key.
There is no limit to the number of columns that you use to create custom message keys. However, it’s best to use the minimum number that are required to specify a unique key.

No default

An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns to be truncated in change event messages if their length exceeds the specified number of characters. Length is specified as a positive integer. A configuration can include multiple properties that specify different lengths. Specify the fully-qualified name for columns by using the following format: <schemaName>.<tableName>.<columnName>.

No default

An optional comma-separated list of regular expressions for masking column names in change event messages by replacing characters with asterisks (*).
Specify the number of characters to replace in the name of the property, for example, column.mask.with.8.chars.
Specify length as a positive integer or zero. Then add regular expressions to the list for each character-based column name where you want to apply a mask.
Use the following format to specify fully-qualified column names: <schemaName>.<tableName>.<columnName>.

The connector configuration can include multiple properties that specify different lengths.

No default

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 are 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 <tableName>.<columnName>, or <schemaName>.<tableName>.<columnName>.

No default

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 are 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 <tableName>.<typeName>, or <schemaName>.<tableName>.<typeName>.
See the list of Oracle-specific data type names.

0

Specifies, in milliseconds, how frequently the connector sends messages to a heartbeat topic.
Use this property to determine whether the connector continues to receive change events from the source database.
It can also be useful to set the property in situations where no change events occur in captured tables for an extended period.
In such a case, although the connector continues to read the redo log, it emits no change event messages, so that the offset in the Kafka topic remains unchanged. Because the connector does not flush the latest system change number (SCN) that it read from the database, the database might retain the redo log files for longer than necessary. If the connector restarts, the extended retention period could result in the connector redundantly sending some change events.
The default value of 0 prevents the connector from sending any heartbeat messages.

No default

Specifies a query that the connector executes on the source database when the connector sends a heartbeat message.

For example:

INSERT INTO test_heartbeat_table (text) VALUES ('test_heartbeat')

The connector runs the query after it emits a heartbeat message.

Set this property and create a heartbeat table to receive the heartbeat messages to resolve situations in which Debezium fails to synchronize offsets on low-traffic databases that are on the same host as a high-traffic database. After the connector inserts records into the configured table, it is able to receive changes from the low-traffic database and acknowledge SCN changes in the database, so that offsets can be synchronized with the broker.

__debezium-heartbeat

Specifies the string that prefixes the name of the topic to which the connector sends heartbeat messages.
The topic is named according to the pattern <heartbeat.topics.prefix>.<serverName>.

No default

Specifies an interval in milliseconds that the connector waits after it starts before it takes a snapshot.
Use this property to prevent snapshot interruptions when you start multiple connectors in a cluster, which might 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 reads table contents in multiple batches of the specified size.

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

Specifies whether field names are normalized to comply with Avro naming requirements. For more information, see Avro naming.

false

Set the property to true if you want Debezium to generate events with transaction boundaries and enriches data events envelope with transaction metadata.

See Transaction Metadata for additional details.

${database.server.name}.transaction

Controls the name of the topic to which the connector sends transaction metadata messages. The placeholder ${database.server.name} can be used for referring to the connector’s logical name; defaults to ${database.server.name}.transaction, for example dbserver1.transaction.

redo_log_catalog

Specifies the mining strategy that controls how Oracle LogMiner builds and uses a given data dictionary for resolving table and column ids to names.

redo_log_catalog:: Writes the data dictionary to the online redo logs causing more archive logs to be generated over time. This also enables tracking DDL changes against captured tables, so if the schema changes frequently this is the ideal choice.

online_catalog:: Uses the database’s current data dictionary to resolve object ids and does not write any extra information to the online redo logs. This allows LogMiner to mine substantially faster but at the expense that DDL changes cannot be tracked. If the captured table(s) schema changes infrequently or never, this is the ideal choice.

memory

The buffer type controls how the connector manages buffering transaction data.

memory - Uses the JVM process' heap to buffer all transaction data. Choose this option if you don’t expect the connector to process a high number of long-running or large transactions. When this option is active, the buffer state is not persisted across restarts. Following a restart, recreate the buffer from the SCN value of the current offset.

infinispan - This option uses an embedded Infinispan cache to buffer transaction data and persist it to disk. Choose this option if you expect the connector to process long-running or large transactions. When this option is active, the buffer state is persisted across restarts. After a restart, there is no need to recreate the buffer. Since the buffer state is persisted across restarts, the buffer does not need to be recreated at restart. The infinispan option requires that you specify a cache file directory. Use the log.mining.buffer.location property to define the location for storing cache files.

No default

The XML configuration for the Infinispan transaction cache. For more information, see Infinispan event buffering.

No default

The XML configuration for the Infinispan events cache. For more information, see Infinispan event buffering.

No default

The XML configuration for the Infinispan processed transactions cache. For more information, see Infinispan event buffering.

No default

The XML configuration for the Infinispan schema changes cache.

false

Specifies whether the buffer state is deleted after the connector stops in a graceful, expected way.

This setting only impacts buffer implementations that persist state across restarts, such as infinispan.
The default behavior is that the buffer state is always retained between restarts.

Set to true only in testing or development environments.

0

The maximum number of milliseconds that a LogMiner session can be active before a new session is used.

For low volume systems, a LogMiner session may consume too much PGA memory when the same session is used for a long period of time. The default behavior is to only use a new LogMiner session when a log switch is detected. By setting this value to something greater than 0, this specifies the maximum number of milliseconds a LogMiner session can be active before it gets stopped and started to deallocate and reallocate PGA memory.

1000

The minimum SCN interval size that this connector attempts to read from redo/archive logs. Active batch size is also increased/decreased by this amount for tuning connector throughput when needed.

100000

The maximum SCN interval size that this connector uses when reading from redo/archive logs.

20000

The starting SCN interval size that the connector uses for reading data from redo/archive logs.

0

The minimum amount of time that the connector sleeps after reading data from redo/archive logs and before starting reading data again. Value is in milliseconds.

3000

The maximum amount of time that the connector ill sleeps after reading data from redo/archive logs and before starting reading data again. Value is in milliseconds.

1000

The starting amount of time that the connector sleeps after reading data from redo/archive logs and before starting reading data again. Value is in milliseconds.

200

The maximum amount of time up or down that the connector uses to tune the optimal sleep time when reading data from logminer. Value is in milliseconds.

10000

The number of content records that the connector fetches from the LogMiner content view.

0

The number of hours in the past from SYSDATE to mine archive logs. When the default setting (0) is used, the connector mines all archive logs.

false

Controls whether or not the connector mines changes from just archive logs or a combination of the online redo logs and archive logs (the default).

Redo logs use a circular buffer that can be archived at any point. In environments where online redo logs are archived frequently, this can lead to LogMiner session failures. In contrast to redo logs, archive logs are guaranteed to be reliable. Set this option to true to force the connector to mine archive logs only. After you set the connector to mine only the archive logs, the latency between an operation being committed and the connector emitting an associated change event might increase. The degree of latency depends on how frequently the database is configured to archive online redo logs.

10000

The number of milliseconds the connector will sleep in between polling to determine if the starting system change number is in the archive logs. If log.mining.archive.log.only.mode is not enabled, this setting is not used.

0

Positive integer value that specifies the number of hours to retain long running transactions between redo log switches. When set to 0, transactions are retained until a commit or rollback is detected.

The LogMiner adapter maintains an in-memory buffer of all running transactions. Because all of the DML operations that are part of a transaction are buffered until a commit or rollback is detected, long-running transactions should be avoided in order to not overflow that buffer. Any transaction that exceeds this configured value is discarded entirely, and the connector does not emit any messages for the operations that were part of the transaction. While this option allows the behavior to be configured on a case-by-case basis, we have plans to enhance this behavior in a future release by means of adding a scalable transaction buffer, (see DBZ-3123).

No default

Specifies the configured Oracle archive destination to use when mining archive logs with LogMiner.

The default behavior automatically selects the first valid, local configured destination. However, you can use a specific destination can be used by providing the destination name, for example, LOG_ARCHIVE_DEST_5.

No default

List of database users to exclude from the LogMiner query. It can be useful to set this property if you want the capturing process to always exclude the changes that specific users make.

1000000

Specifies a value that the connector compares to the difference between the current and previous SCN values to determine whether an SCN gap exists. If the difference between the SCN values is greater than the specified value, and the time difference is smaller than log.mining.scn.gap.detection.time.interval.max.ms then an SCN gap is detected, and the connector uses a mining window larger than the configured maximum batch.

20000

Specifies a value, in milliseconds, that the connector compares to the difference between the current and previous SCN timestamps to determine whether an SCN gap exists. If the difference between the timestamps is less than the specified value, and the SCN delta is greater than log.mining.scn.gap.detection.gap.size.min, then an SCN gap is detected and the connector uses a mining window larger than the configured maximum batch.

false

Controls whether or not large object (CLOB or BLOB) column values are emitted in change events.

By default, change events have large object columns, but the columns contain no values. There is a certain amount of overhead in processing and managing large object column types and payloads. To capture large object values and serialized them in change events, set this option to true.

Use of large object data types is a Technology Preview feature.

__debezium_unavailable_value

Specifies the constant that the connector provides to indicate that the original value is unchanged and not provided by the database.

No default

A comma-separated list of Oracle Real Application Clusters (RAC) node host names or addresses. This field is required to enable Oracle RAC support. Specify the list of RAC nodes by using one of the following methods:

  • Specify a value for database.port, and use the specified port value for each address in the rac.nodes list. For example:

    database.port=1521
    rac.nodes=192.168.1.100,192.168.1.101
  • Specify a value for database.port, and override the default port for one or more entries in the list. The list can include entries that use the default database.port value, and entries that define their own unique port values. For example:

    database.port=1521
    rac.nodes=192.168.1.100,192.168.1.101:1522

If you supply a raw JDBC URL for the database by using the database.url property, instead of defining a value for database.port, each RAC node entry must explicitly specify a port value.

No default

A comma-separated list of the operation types that you want the connector to skip during streaming. You can configure the connector to skip the following types of operations:

  • c (insert/create)

  • u (update)

  • d (delete)

  • t (truncate)

By default, no operations are skipped.

No default value

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

1024

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

Debezium Oracle connector database history configuration properties

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

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

Table 11. Connector database history configuration properties
Property Default Description

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

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

100

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

3000

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

4

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

false

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

Deprecated and scheduled for removal in a future release; use database.history.store.only.captured.tables.ddl instead.

false

A Boolean value that specifies whether the connector should record all DDL statements

true records only those DDL statements that are relevant to tables whose changes are being captured by Debezium. Set to true with care because missing data might become necessary if you change which tables have their changes captured.

The safe default is false.

false

A Boolean value that specifies whether the connector should record all DDL statements

true records only those DDL statements that are relevant to tables whose changes are being captured by Debezium. Set to true with care because missing data might become necessary if you change which tables have their changes captured.

The safe default is false.

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


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

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

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

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

Debezium Oracle connector pass-through database driver configuration properties

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

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

Monitoring

The Debezium Oracle connector provides three metric types in addition to the built-in support for JMX metrics that Apache Zookeeper, Apache 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.oracle:type=connector-metrics,context=snapshot,server=<oracle.server.name>.

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

The following table lists the shapshot metrics that are available.

Attributes Type Description

string

The last snapshot event that the connector has read.

long

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

long

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

long

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

string[]

The list of tables that are captured by the connector.

int

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

int

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

int

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

int

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

boolean

Whether the snapshot was started.

boolean

Whether the snapshot was aborted.

boolean

Whether the snapshot completed.

long

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

Map<String, Long>

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

long

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

long

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

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

Attributes Type Description

string

The identifier of the current snapshot chunk.

string

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

string

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

string

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

string

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

Streaming Metrics

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

The following table lists the streaming metrics that are available.

Attributes Type Description

string

The last streaming event that the connector has read.

long

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

long

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

long

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

long

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

long

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

long

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

string[]

The list of tables that are captured by the connector.

int

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

int

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

boolean

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

long

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

long

The number of processed transactions that were committed.

Map<String, String>

The coordinates of the last received event.

string

Transaction identifier of the last processed transaction.

long

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

long

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

The Debezium Oracle connector also provides the following additional streaming metrics:

Table 12. Descriptions of additional streaming metrics
Attributes Type Description

string

The most recent system change number that has been processed.

string

The oldest system change number in the transaction buffer.

string

The last committed system change number from the transaction buffer.

string

The system change number currently written to the connector’s offsets.

string[]

Array of the log files that are currently mined.

long

The minimum number of logs specified for any LogMiner session.

long

The maximum number of logs specified for any LogMiner session.

string[]

Array of the current state for each mined logfile with the format filename|status.

int

The number of times the database has performed a log switch for the last day.

long

The number of DML operations observed in the last LogMiner session query.

long

The maximum number of DML operations observed while processing a single LogMiner session query.

long

The total number of DML operations observed.

long

The total number of LogMiner session query (aka batches) performed.

long

The duration of the last LogMiner session query’s fetch in milliseconds.

long

The maximum duration of any LogMiner session query’s fetch in milliseconds.

long

The duration for processing the last LogMiner query batch results in milliseconds.

long

The time in milliseconds spent parsing DML event SQL statements.

long

The duration in milliseconds to start the last LogMiner session.

long

The longest duration in milliseconds to start a LogMiner session.

long

The total duration in milliseconds spent by the connector starting LogMiner sessions.

long

The minimum duration in milliseconds spent processing results from a single LogMiner session.

long

The maximum duration in milliseconds spent processing results from a single LogMiner session.

long

The total duration in milliseconds spent processing results from LogMiner sessions.

long

The total duration in milliseconds spent by the JDBC driver fetching the next row to be processed from the log mining view.

long

The total number of rows processed from the log mining view across all sessions.

int

The number of entries fetched by the log mining query per database round-trip.

long

The number of milliseconds the connector sleeps before fetching another batch of results from the log mining view.

long

The maximum number of rows/second processed from the log mining view.

long

The average number of rows/second processed from the log mining.

long

The average number of rows/second processed from the log mining view for the last batch.

long

The number of connection problems detected.

int

The number of hours that transactions are retained by the connector’s in-memory buffer without being committed or rolled back before being discarded. See log.mining.transaction.retention for more details.

long

The number of current active transactions in the transaction buffer.

long

The number of committed transactions in the transaction buffer.

long

The number of rolled back transactions in the transaction buffer.

long

The average number of committed transactions per second in the transaction buffer.

long

The number of registered DML operations in the transaction buffer.

long

The time difference in milliseconds between when a change occurred in the transaction logs and when its added to the transaction buffer.

long

The maximum time difference in milliseconds between when a change occurred in the transaction logs and when its added to the transaction buffer.

long

The minimum time difference in milliseconds between when a change occurred in the transaction logs and when its added to the transaction buffer.

string[]

An array of abandoned transaction identifiers removed from the transaction buffer due to their age. See log.mining.transaction.retention.hours for details.

string[]

An array of transaction identifiers that have been mined and rolled back in the transaction buffer.

long

The duration of the last transaction buffer commit operation in milliseconds.

long

The duration of the longest transaction buffer commit operation in milliseconds.

int

The number of errors detected.

int

The number of warnings detected.

int

The number of times that the system change number was checked for advancement and remains unchanged. A high value can indicate that a long-running transactions is ongoing and is preventing the connector from flushing the most recently processed system change number to the connector’s offsets. When conditions are optimal, the value should be close to or equal to 0.

int

The number of DDL records that have been detected but could not be parsed by the DDL parser. This should always be 0; however when allowing unparsable DDL to be skipped, this metric can be used to determine if any warnings have been written to the connector logs.

long

The current mining session’s user global area (UGA) memory consumption in bytes.

long

The maximum mining session’s user global area (UGA) memory consumption in bytes across all mining sessions.

long

The current mining session’s process global area (PGA) memory consumption in bytes.

long

The maximum mining session’s process global area (PGA) memory consumption in bytes across all mining sessions.

Schema History Metrics

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

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

Attributes Type Description

string

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

long

The time in epoch seconds at what recovery has started.

long

The number of changes that were read during recovery phase.

long

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

long

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

long

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

string

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

string

The string representation of the last applied change.

Surrogate schema evolution

The Oracle connector automatically tracks and applies table schema changes by parsing DDL from the redo logs. If the DDL parser encounters an incompatible statement, if needed, the connector provides an alternative way to apply the schema change.

By default, the connector stops when it encounters a DDL statement that it cannot parse. You can use Debezium signaling to trigger the update of the database schema from such DDL statements.

The type of the schema update action is schema-changes. This action updates the schema of all tables enumerated in the signal parameters. The message does not contain the update to the schema. Instead, it contains the complete new schema structure.

Table 13. Action parameters
Name Description

database

The name of the Oracle database.

schema

The name of the schema where changes are applied.

changes

An array containing the requested schema updates.

changes.type

Type of the schema change, usually ALTER

changes.id

The fully-qualified name of the table

changes.table

The fully-qualified name of the table

changes.table.defaultCharsetName

The character set name used for the table if different from database default

changes.table.primaryKeyColumnNames

Array with the name of columns composing the primary key

changes.table.columns

Array with the column metadata

…​columns.name

The name of the column

…​columns.jdbcType

The JDBC type of the column as defined at JDBC API

…​columns.typeName

The name of the column type

…​columns.typeExpression

The full column type definition

…​columns.charsetName

The column character set if different from the default

…​columns.length

The length/size constraint of the column

…​columns.scale

The scale of numeric column

…​columns.position

The position of the column in the table starting with 1

…​columns.optional

Boolean true if column value is not mandatory

…​columns.autoIncremented

Boolean true if column value is automatically calculated from a sequence

…​columns.generated

Boolean true if column value is automatically calculated

After the schema-changes signal is inserted, the connector must be restarted with an altered configuration that includes specifying the database.history.skip.unparseable.ddl option as true. After the connector’s commit SCN advances beyond the DDL change, to prevent unparseable DDL statements from being skipped unexpectedly, return the connector configuration to its previous state.

Table 14. Example of a logging record
Column Value

id

924e3ff8-2245-43ca-ba77-2af9af02fa07

type

schema-changes

data

{
   "database":"ORCLPDB1",
   "schema":"DEBEZIUM",
   "changes":[
      {
         "type":"ALTER",
         "id":"\"ORCLPDB1\".\"DEBEZIUM\".\"CUSTOMER\"",
         "table":{
            "defaultCharsetName":null,
            "primaryKeyColumnNames":[
               "ID",
               "NAME"
            ],
            "columns":[
               {
                  "name":"ID",
                  "jdbcType":2,
                  "typeName":"NUMBER",
                  "typeExpression":"NUMBER",
                  "charsetName":null,
                  "length":9,
                  "scale":0,
                  "position":1,
                  "optional":false,
                  "autoIncremented":false,
                  "generated":false
               },
               {
                  "name":"NAME",
                  "jdbcType":12,
                  "typeName":"VARCHAR2",
                  "typeExpression":"VARCHAR2",
                  "charsetName":null,
                  "length":1000,
                  "position":2,
                  "optional":true,
                  "autoIncremented":false,
                  "generated":false
               },
               {
                  "name":"SCORE",
                  "jdbcType":2,
                  "typeName":"NUMBER",
                  "typeExpression":"NUMBER",
                  "charsetName":null,
                  "length":6,
                  "scale":2,
                  "position":3,
                  "optional":true,
                  "autoIncremented":false,
                  "generated":false
               },
               {
                  "name":"REGISTERED",
                  "jdbcType":93,
                  "typeName":"TIMESTAMP(6)",
                  "typeExpression":"TIMESTAMP(6)",
                  "charsetName":null,
                  "length":6,
                  "position":4,
                  "optional":true,
                  "autoIncremented":false,
                  "generated":false
               }
            ]
         }
      }
   ]
}

XStreams support

The Debezium Oracle connector by default ingests changes using native Oracle LogMiner. The connector can be toggled to use Oracle XStream instead. To configure the connector to use Oracle XStream, you must apply specific database and connector configurations that differ from those that you use with LogMiner.

Prerequisites
  • To use the XStream API, you must have a license for the GoldenGate product. Installing GoldenGate is not required.

Preparing the Database

Configuration needed for Oracle XStream
ORACLE_SID=ORCLCDB dbz_oracle sqlplus /nolog

CONNECT sys/top_secret AS SYSDBA
alter system set db_recovery_file_dest_size = 5G;
alter system set db_recovery_file_dest = '/opt/oracle/oradata/recovery_area' scope=spfile;
alter system set enable_goldengate_replication=true;
shutdown immediate
startup mount
alter database archivelog;
alter database open;
-- Should show "Database log mode: Archive Mode"
archive log list

exit;

In addition, supplemental logging must be enabled for captured tables or the database in order for data changes to capture the before state of changed database rows. The following illustrates how to configure this on a specific table, which is the ideal choice to minimize the amount of information captured in the Oracle redo logs.

ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS;

Creating XStream users for the connector

The Debezium Oracle connector requires that users accounts be set up with specific permissions so that the connector can capture change events. The following briefly describes these user configurations using a multi-tenant database model.

Creating an XStream Administrator user
sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba
  CREATE TABLESPACE xstream_adm_tbs DATAFILE '/opt/oracle/oradata/ORCLCDB/xstream_adm_tbs.dbf'
    SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
  exit;

sqlplus sys/top_secret@//localhost:1521/ORCLPDB1 as sysdba
  CREATE TABLESPACE xstream_adm_tbs DATAFILE '/opt/oracle/oradata/ORCLCDB/ORCLPDB1/xstream_adm_tbs.dbf'
    SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
  exit;

sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba
  CREATE USER c##dbzadmin IDENTIFIED BY dbz
    DEFAULT TABLESPACE xstream_adm_tbs
    QUOTA UNLIMITED ON xstream_adm_tbs
    CONTAINER=ALL;

  GRANT CREATE SESSION, SET CONTAINER TO c##dbzadmin CONTAINER=ALL;

  BEGIN
     DBMS_XSTREAM_AUTH.GRANT_ADMIN_PRIVILEGE(
        grantee                 => 'c##dbzadmin',
        privilege_type          => 'CAPTURE',
        grant_select_privileges => TRUE,
        container               => 'ALL'
     );
  END;
  /

  exit;
Creating the connector’s XStream user
sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba
  CREATE TABLESPACE xstream_tbs DATAFILE '/opt/oracle/oradata/ORCLCDB/xstream_tbs.dbf'
    SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
  exit;

sqlplus sys/top_secret@//localhost:1521/ORCLPDB1 as sysdba
  CREATE TABLESPACE xstream_tbs DATAFILE '/opt/oracle/oradata/ORCLCDB/ORCLPDB1/xstream_tbs.dbf'
    SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
  exit;

sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba
  CREATE USER c##dbzuser IDENTIFIED BY dbz
    DEFAULT TABLESPACE xstream_tbs
    QUOTA UNLIMITED ON xstream_tbs
    CONTAINER=ALL;

  GRANT CREATE SESSION TO c##dbzuser CONTAINER=ALL;
  GRANT SET CONTAINER TO c##dbzuser CONTAINER=ALL;
  GRANT SELECT ON V_$DATABASE to c##dbzuser CONTAINER=ALL;
  GRANT FLASHBACK ANY TABLE TO c##dbzuser CONTAINER=ALL;
  GRANT SELECT_CATALOG_ROLE TO c##dbzuser CONTAINER=ALL;
  GRANT EXECUTE_CATALOG_ROLE TO c##dbzuser CONTAINER=ALL;
  exit;

Create an XStream Outbound Server

Create an XStream Outbound server (given the right privileges, this might be done automatically by the connector going forward, see DBZ-721):

Create an XStream Outbound Server
sqlplus c##dbzadmin/dbz@//localhost:1521/ORCLCDB
DECLARE
  tables  DBMS_UTILITY.UNCL_ARRAY;
  schemas DBMS_UTILITY.UNCL_ARRAY;
BEGIN
    tables(1)  := NULL;
    schemas(1) := 'debezium';
  DBMS_XSTREAM_ADM.CREATE_OUTBOUND(
    server_name     =>  'dbzxout',
    table_names     =>  tables,
    schema_names    =>  schemas);
END;
/
exit;

When setting up an XStream Outbound Server to capture changes from a pluggable database, the source_container_name parameter should be provided specifying the pluggable database name.

Configure the XStream user account to connect to the XStream Outbound Server
sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba
BEGIN
  DBMS_XSTREAM_ADM.ALTER_OUTBOUND(
    server_name  => 'dbzxout',
    connect_user => 'c##dbzuser');
END;
/
exit;

A single XStream Outbound server cannot be shared by multiple Debezium Oracle connectors. Each connector requires a unique XStream Outbound connector to be configured.

Configuring the XStream adapter

By default, Debezium uses Oracle LogMiner to ingest change events from Oracle. You can adjust the connector configuration to enable the connector to use the Oracle XStreams adapter.

The following configuration example adds the properties database.connection.adapter and database.out.server.name to enable the connector to use the XStream API implementation.

{
    "name": "inventory-connector",
    "config": {
        "connector.class" : "io.debezium.connector.oracle.OracleConnector",
        "tasks.max" : "1",
        "database.server.name" : "server1",
        "database.hostname" : "<oracle ip>",
        "database.port" : "1521",
        "database.user" : "c##dbzuser",
        "database.password" : "dbz",
        "database.dbname" : "ORCLCDB",
        "database.pdb.name" : "ORCLPDB1",
        "database.history.kafka.bootstrap.servers" : "kafka:9092",
        "database.history.kafka.topic": "schema-changes.inventory",
        "database.connection.adapter": "xstream",
        "database.out.server.name" : "dbzxout"
    }
}

Obtaining the Oracle JDBC driver and XStream API files

The Debezium Oracle connector requires the Oracle JDBC driver (ojdbc8.jar) to connect to Oracle databases. If the connector uses XStream to access the database, you must also have the XStream API (xstreams.jar). Licensing requirements prohibit Debezium from including these files in the Oracle connector archive. However, the required files are available for free download as part of the Oracle Instant Client. The following steps describe how to download the Oracle Instant Client and extract the required files.

Procedure
  1. From a browser, download the Oracle Instant Client package for your operating system.

  2. Extract the archive, and then open the instantclient_<version> directory.

    For example:

    instantclient_21_1/
    ├── adrci
    ├── BASIC_LITE_LICENSE
    ├── BASIC_LITE_README
    ├── genezi
    ├── libclntshcore.so -> libclntshcore.so.21.1
    ├── libclntshcore.so.12.1 -> libclntshcore.so.21.1
    
    ...
    
    ├── ojdbc8.jar
    ├── ucp.jar
    ├── uidrvci
    └── xstreams.jar
  3. Copy the ojdbc8.jar and xstreams.jar files, and add them to the <kafka_home>/libs directory, for example, kafka/libs.

  4. Create an environment variable, LD_LIBRARY_PATH, and set its value to the path to the Instant Client directory, for example:

    LD_LIBRARY_PATH=/path/to/instant_client/

XStream connector properties

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

Property

Default

Description

No default

Name of the XStream outbound server configured in the database.

XStream and DBMS_LOB

Oracle provides a database package called DBMS_LOB that consists of a collection of programs to operate on BLOB, CLOB, and NCLOB columns. Most of these programs manipulate the LOB column in totality, however, one program, WRITEAPPEND, is capable of manipulating a subset of the LOB data buffer.

When using XStream, WRITEAPPEND emits a logical change record (LCR) event for each invocation of the program. These LCR events are not combined into a single change event like they are when using the Oracle LogMiner adapter, and so consumers of the topic should be prepared to receive events with partial column values. This diverged behavior is captured in DBZ-4741 and will be addressed in a future release.

Behavior when things go wrong

Debezium is a distributed system that captures all changes in multiple upstream databases; it never misses or loses an event. When the system is operating normally or being managed carefully then Debezium provides exactly once delivery of every change event record.

If a fault occurs, Debezium does not lose any events. However, while it is recovering from the fault, it might repeat some change events. In these abnormal situations, Debezium, like Kafka, provides at least once delivery of change events.

The rest of this section describes how Debezium handles various kinds of faults and problems.

Logs do not contain offset, perform a new snapshot

In some cases, after the Debezium Oracle connector restarts, it reports the following error:

Online REDO LOG files or archive logs do not contain the offset scn xxxxxxx. Please perform a new snapshot.

After the connector examines the redo and archive logs, if it cannot find the SCN that is recorded in the connector offsets, it returns the preceding error. Because the connector uses the SCN to determine where to resume processing, if the expected SCN if not found, a new snapshot must be completed.

You might find that the V$ARCHIVED_LOG table contains a record with an SCN that matches the expected range. However, the record might not be available for mining. To be available for mining, a record must include a filename in the NAME column, a value of NO in the DELETED column, and a value of A (available) in the STATUS column. If a record does not match any of these criteria, it is considered incomplete and cannot be mined.

At a minimum, archive logs must be retained for as long as the longest downtime window of the connector.

Records that have no value in the NAME column no longer exist in the file system. In such records, the value of the DELETED field is set to YES, and the STATUS field is set to D to indicate that the log is deleted.

ORA-25191 - Cannot reference overflow table of an index-organized table

Oracle might issue this error during the snapshot phase when encountering an index-organized table (IOT). This error means that the connector has attempted to execute an operation that must be executed against the parent index-organized table that contains the specified overflow table.

To resolve this, the IOT name used in the SQL operation should be replaced with the parent index-organized table name. To determine the parent index-organized table name, use the following SQL:

SELECT IOT_NAME
  FROM DBA_TABLES
 WHERE OWNER='<tablespace-owner>'
   AND TABLE_NAME='<iot-table-name-that-failed>'

The connector’s table.include.list or table.exclude.list configuration options should then be adjusted to explicitly include or exclude the appropriate tables to avoid the connector from attempting to capture changes from the child index-organized table.

ORA-04036: PGA memory used by the instance exceeds PGA_AGGREGATE_LIMIT

Oracle might report this error when Debezium connects to a database in which changes occur infrequently. The Debezium connector starts an Oracle LogMiner session and reuses this session until a log switch is detected. The reuse is both a performance and resource utilization optimization; however, a long-running mining session can cause high Program Global Area (PGA) memory usage.

If your redo log switches infrequently, you can avoid the ORA-04036 error by specifying how frequently Oracle switches logs. A log switch causes the connector to restart the mining session, thereby avoiding high PGA memory usage. The following configuration forces Oracle to switch log files every 20 minutes if a log switch does not occur during that interval:

ALTER SYSTEM SET archive_lag_target=1200 scope=both;

Running the preceding query requires specific administrative privileges. Coordinate with your database administrator to implement the change.

As alternative to adjusting the Oracle ARCHIVE_LAG_TARGET parameter, you can limit the duration of an Oracle LogMiner session by setting the connector configuration option log.mining.session.max.ms. The log.mining.session.max.ms option causes the LogMiner session to restart regularly, whether or not the switch to a new database log occurs.

LogMiner adapter does not capture changes made by SYS or SYSTEM

Oracle uses the SYS and SYSTEM accounts to carry out many internal changes. When the Debezium Oracle connector fetches changes from LogMiner, it automatically filters changes that originate from these administrator accounts. To ensure that the connector emit event records when you change a table, never use the SYS or SYSTEM user accounts to modify the table.

Connector stops capturing changes from Oracle on AWS

Due to the fixed idle timeout of 350 seconds on the AWS Gateway Load Balancer, JDBC calls that require more than 350 seconds to complete can hang indefinitely.

In situations where calls to the Oracle LogMiner API take more than 350 seconds to complete, a timeout can be triggered, causing the AWS Gateway Load Balancer to hang. For example, such timeouts can occur when a LogMiner session that processes large amounts of data runs concurrently with Oracle’s periodic checkpointing task.

To prevent timeouts from occurring on the AWS Gateway Load Balancer, enable keep-alive packets from the kafka Connect or Debezium Server environment, by performing the following task as the root user or as a super-user in the environment that hosts the connector:

  1. From a terminal, run the following command:

    sysctl -w net.ipv4.tcp_keepalive_time=60
  2. Edit /etc/sysctl.conf and set the value of the following variable as shown:

    net.ipv4.tcp_keepalive_time=60
  3. Reconfigure the Debezium for Oracle connector to use the database.url property rather than database.hostname and add the (ENABLE=broken) Oracle connect string descriptor as shown in the following example:

    database.url=jdbc:oracle:thin:username/password!@(DESCRIPTION=(ENABLE=broken)(ADDRESS_LIST=(ADDRESS=(PROTOCOL=TCP)(Host=hostname)(Port=port)))(CONNECT_DATA=(SERVICE_NAME=serviceName)))

The preceding steps configure the TCP network stack to send keep-alive packets every 60 seconds. As a result, the AWS Gateway Load Balancer does not timeout when JDBC calls to the LogMiner API take more than 350 seconds to complete, enabling the connector to continue to read changes from the database’s transaction logs.

Connection fails with error ORA-01882: timezone region not found

When you start a connector that the Oracle JDBC driver (ojdbc8.jar) to access an Oracle 11g database, the connector might report the following error:

Caused by: java.sql.SQLException: ORA-00604: error occurred at recursive SQL level 1
ORA-01882: timezone region not found

To prevent the preceding error from occurring, add the following setting to the connector configuration: database.oracle.jdbc.timezoneAsRegion=false.