Debezium Connector for Oracle
Debezium’s Oracle Connector can monitor and record all of the row-level changes in the databases on an Oracle server.
Most notably, the connector does not yet provide full-blown support of changes to the structure of captured tables (e.g. ALTER TABLE…
) after the initial snapshot has been completed
(see DBZ-718).
It is supported though to capture tables newly added while the connector is running
(provided the new table’s name matches the connector’s filter configuration).
For table schema changes an auxilliary solution could be used.
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.
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, and uses metadata.
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
.
When the snapshot mode is set to the default, the connector completes the following tasks to create a snapshot:
-
Determines the tables to be captured
-
Obtains a
EXCLUSIVE 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. -
Reads the current system change number (SCN) position from the server’s redo log.
-
Captures the structure of all relevant tables.
-
Releases the locks obtained in Step 2.
-
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 aREAD
event for each row, and then writes the event records to the table-specific Kafka topic. -
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.
Setting | Description |
---|---|
|
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. |
|
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 |
Topic names
The default behavior is that a Debezium Oracle connector writes events for all INSERT
, UPDATE
, and DELETE
operations in one table to one Kafka topic.
The Kafka topic naming convention is as follows:
serverName.schemaName.tableName
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
Schema change topic
The Debezium Oracle connector stores the history of schema changes in a database history topic.
This topic reflects an internal connector state and you should not use it directly.
Applications that require notifications about schema changes should obtain the information from the public schema change topic.
The connector writes schema change events to a Kafka topic named <serverName>
, where serverName
is the name of the connector that is specified in the database.server.name
configuration property.
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.
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. |
The message contains a logical representation of the table schema.
The following example shows a typical schema change message in JSON format:
{
"schema": {
...
},
"payload": {
"source": {
"version": "1.5.4.Final",
"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
},
"databaseName": "ORCLPDB1", (1)
"schemaName": "DEBEZIUM", (1)
"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\" ", (2)
"tableChanges": [ (3)
{
"type": "CREATE", (4)
"id": "\"ORCLPDB1\".\"DEBEZIUM\".\"CUSTOMERS\"", (5)
"table": { (6)
"defaultCharsetName": null,
"primaryKeyColumnNames": [ (7)
"ID"
],
"columns": [ (8)
{
"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
}
]
}
}
]
}
}
Item | Field name | Description |
---|---|---|
1 |
|
Identifies the database and the schema that contains the change. |
2 |
|
This field contains the DDL that is responsible for the schema change. |
3 |
|
An array of one or more items that contain the schema changes generated by a DDL command. |
4 |
|
Describes the kind of change. The value is one of the following:
|
5 |
|
Full identifier of the table that was created, altered, or dropped. |
6 |
|
Represents table metadata after the applied change. |
7 |
|
List of columns that compose the table’s primary key. |
8 |
|
Metadata for each column in the changed table. |
Messages that the connector sends to the schema change topic use a message key that is equal to the name of the database that contains the schema change.
In the following example, the payload
field contains the key:
{
"schema": {
"type": "struct",
"fields": [
{
"type": "string",
"optional": false,
"field": "databaseName"
}
],
"optional": false,
"name": "io.debezium.connector.oracle.SchemaChangeKey"
},
"payload": {
"databaseName": "ORCLPDB1"
}
}
Transaction Metadata
Debezium can generate events that represent transaction metadata boundaries and that enrich data change event messages.
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
orEND
id
-
String representation of unique transaction identifier.
event_count
(forEND
events)-
Total number of events emmitted by the transaction.
data_collections
(forEND
events)-
An array of pairs of
data_collection
andevent_count
that provides number of events emitted by changes originating from the given data collection.
The following example shows a typical transaction boundary message:
{
"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
}
]
}
The transaction events are written to the topic named <database.server.name>.transaction
.
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"
}
}
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 |
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
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
Like the message key, the value of a change event message has a schema section and payload section. The payload section of every change event value produced by the Oracle connector has an envelope structure with the following fields:
op
-
A mandatory field that contains a string value describing the type of operation. Values for the Oracle connector are
c
for create (or insert),u
for update,d
for delete, andr
for read (in the case of a snapshot). before
-
An optional field that, if present, contains the state of the row before the event occurred. The structure is described by the
server1.INVENTORY.CUSTOMERS.Value
Kafka Connect schema, which theserver1
connector uses for all rows in theinventory.customers
table.
Whether or not this field and its elements are available is highly dependent on the Supplemental Logging configuration applying to the table. |
after
-
An optional field that if present contains the state of the row after the event occurred. The structure is described by the same
server1.INVENTORY.CUSTOMERS.Value
Kafka Connect schema used inbefore
. source
-
A mandatory field that contains a structure describing the source metadata for the event, which in the case of Oracle contains these fields: the Debezium version, the connector name, whether the event is part of an ongoing snapshot or not, the transaction id (not while snapshotting), the SCN of the change, and a timestamp representing the point in time when the record was changed in the source database (during snapshotting, this is the point in time of snapshotting).
The |
ts_ms
-
An optional field that, if present, contains the time (using the system clock in the JVM running the Kafka Connect task) at which the connector processed the event.
And of course, the schema portion of the event message’s value contains a schema that describes this envelope structure and the nested fields within it.
create events
Let’s look at what a create event value might look like for our customers
table:
{
"schema": {
"type": "struct",
"fields": [
{
"type": "struct",
"fields": [
{
"type": "int32",
"optional": false,
"field": "ID"
},
{
"type": "string",
"optional": false,
"field": "FIRST_NAME"
},
{
"type": "string",
"optional": false,
"field": "LAST_NAME"
},
{
"type": "string",
"optional": false,
"field": "EMAIL"
}
],
"optional": true,
"name": "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": "1.5.4.Final",
"name": "server1",
"ts_ms": 1520085154000,
"txId": "6.28.807",
"scn": "2122185",
"commit_scn": "2122185",
"snapshot": false
},
"op": "c",
"ts_ms": 1532592105975
}
}
Examining the schema
portion of the preceding event’s value, we can see how the following schema are defined:
-
The envelope
-
The
source
structure (which is specific to the Oracle connector and reused across all events). -
The table-specific schemas for the
before
andafter
fields.
The names of the schemas for the |
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 they describe. This is true, because the JSON representation must include the schema and the payload portions of the message. You can use the Avro Converter to significantly decrease the size of the messages that the connector writes to Kafka topics. |
update events
The value of an update change event on this table has the same schema as the create event. The payload uses the same structure, but it holds different values. Here’s an example:
{
"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": "1.5.4.Final",
"name": "server1",
"ts_ms": 1520085811000,
"txId": "6.9.809",
"scn": "2125544",
"commit_scn": "2125544",
"snapshot": false
},
"op": "u",
"ts_ms": 1532592713485
}
}
Comparing the value of the update event to the create (insert) event, notice the following differences in the payload
section:
-
The
op
field value is nowu
, signifying that this row changed because of an update -
The
before
field now has the state of the row with the values before the database commit -
The
after
field now has the updated state of the row, and here was can see that theEMAIL
value is nowanne@example.com
. -
The
source
field structure has the same fields as before, but the values are different since this event is from a different position in the redo log. -
The
ts_ms
shows the timestamp that Debezium processed this 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:
|
delete events
So far we’ve seen samples of create and update events.
Now, let’s look at the value of a delete event for the same table. As is the case with create and update events, for a delete
event, the schema
portion of the value is exactly the same:
{
"schema": { ... },
"payload": {
"before": {
"ID": 1004,
"FIRST_NAME": "Anne",
"LAST_NAME": "Kretchmar",
"EMAIL": "anne@example.com"
},
"after": null,
"source": {
"version": "1.5.4.Final",
"name": "server1",
"ts_ms": 1520085153000,
"txId": "6.28.807",
"scn": "2122184",
"commit_scn": "2122184",
"snapshot": false
},
"op": "d",
"ts_ms": 1532592105960
}
}
If we look at the payload
portion, we see a number of differences compared with the create or update event payloads:
-
The
op
field value is nowd
, signifying that this row was deleted -
The
before
field now has the state of the row that was deleted with the database commit. -
The
after
field is null, signifying that the row no longer exists -
The
source
field structure has many of the same values as before, except thets_ms
,scn
andtxId
fields have changed -
The
ts_ms
shows the timestamp that Debezium processed this event.
This event gives a consumer all kinds of information that it can use to process the removal of this row.
The 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 listed above still works with log compaction, since Kafka can still remove all earlier messages with that 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
.
Data type mappings
To represent changes that occur in a table rows, the Debezium Oracle connector emits change events that are structured like the table in which the rows exists. The event contains a field for each column value. Column values are represented according to the Oracle data type of the column. The following sections describe how the connector maps oracle data types to a literal type and a semantic type in event fields.
- literal type
-
Describes how the value is literally represented using Kafka Connect schema types:
INT8
,INT16
,INT32
,INT64
,FLOAT32
,FLOAT64
,BOOLEAN
,STRING
,BYTES
,ARRAY
,MAP
, andSTRUCT
. - semantic type
-
Describes how the Kafka Connect schema captures the meaning of the field using the name of the Kafka Connect schema for the field.
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.
Oracle Data Type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
|
|
n/a |
Binary and Character LOB types
The following table describes how the connector maps binary and character LOB types.
Oracle Data Type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
n/a |
This data type is not supported. |
|
n/a |
This data type is not supported. |
|
n/a |
This data type is not supported. |
|
n/a |
This data type is not supported. |
|
n/a |
This data type is not supported. |
|
n/a |
This data type is not supported. |
Numeric types
The following table describes how the connector maps numeric types.
Oracle Data Type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
n/a |
|
|
n/a |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
Boolean types
Oracle does not natively have 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.
The operator can configure the out-of-the-box NumberOneToBooleanConverter
custom converter that would either map all NUMBER(1)
columns to a BOOLEAN
or if the selector
parameter is set,
then a subset of columns could be enumerated using a comma-separated list of regular expressions.
Following is an example configuration:
converters=boolean
boolean.type=io.debezium.connector.oracle.converters.NumberOneToBooleanConverter
boolean.selector=.*MYTABLE.FLAG,.*.IS_ARCHIVED
Decimal types
The setting of the Oracle connector configuration property, decimal.handling.mode
determines how the connector maps decimal types.
When the decimal.handling.mode
property is set to precise
, the connector uses Kafka Connect org.apache.kafka.connect.data.Decimal
logical type for all DECIMAL
and NUMERIC
columns.
This is the default mode.
However, when the decimal.handling.mode
property is set to double
, the connector represents the values as Java double values with schema type FLOAT64
.
You can also set the decimal.handling.mode
configuration property to use the string
option.
When the property is set to string
, the connector represents DECIMAL
and NUMERIC
values as their formatted string representation with schema type STRING
.
Temporal types
Other than Oracle’s INTERVAL
, TIMESTAMP WITH TIME ZONE
and TIMESTAMP WITH LOCAL TIME ZONE
data types, the other temporal types depend on the value of the time.precision.mode
configuration property.
When the time.precision.mode
configuration property is set to adaptive
(the default), then the connector 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 |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
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 |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
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.
Preparing the database
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;
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;
Minimal supplemental logging must be enabled at the database level and can be configured as follows.
ALTER DATABASE ADD SUPPLEMENTAL LOG DATA;
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 does not capture database changes made by the |
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;
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 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;
GRANT SELECT ANY TRANSACTION TO c##dbzuser CONTAINER=ALL;
GRANT LOGMINING TO c##dbzuser CONTAINER=ALL;
GRANT CREATE TABLE TO c##dbzuser CONTAINER=ALL;
GRANT LOCK ANY TABLE TO c##dbzuser CONTAINER=ALL;
GRANT ALTER ANY TABLE TO c##dbzuser CONTAINER=ALL;
GRANT CREATE SEQUENCE TO c##dbzuser CONTAINER=ALL;
GRANT EXECUTE ON DBMS_LOGMNR TO c##dbzuser CONTAINER=ALL;
GRANT EXECUTE ON DBMS_LOGMNR_D TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$LOG TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$LOG_HISTORY TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$LOGMNR_LOGS TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$LOGMNR_CONTENTS TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$LOGMNR_PARAMETERS TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$LOGFILE TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$ARCHIVED_LOG TO c##dbzuser CONTAINER=ALL;
GRANT SELECT ON V_$ARCHIVE_DEST_STATUS TO c##dbzuser CONTAINER=ALL;
exit;
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.
-
Apache ZooKeeper, Apache Kafka, and Kafka Connect are installed.
-
Oracle Database is installed, and is configured to work with the Debezium connector.
-
You have a copies of the Oracle JDBC driver and the XStream API JAR.
Due to licensing requirements, the Debezium Oracle connector does not ship with the Oracle JDBC driver or XStream API files. You must download these files directly from Oracle and add them to your environment. For more information, see Obtaining the Oracle JDBC driver and XStream API files.
-
Download the Debezium Oracle connector plug-in archive.
-
Extract the files into your Kafka Connect environment.
-
Add the directory with the JAR files to Kafka Connect’s
plugin.path
. -
Configure the connector and add the configuration to your Kafka Connect cluster.
-
Restart your Kafka Connect process to pick up the new JAR files.
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 XStreams 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.
-
From a browser, download the Oracle Instant Client package for your operating system.
-
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
-
Copy the
ojdbc8.jar
andxstreams.jar
files, and add them to the<kafka_home>/libs
directory, for example,kafka/libs
.In environments that use the Oracle LogMiner implementation, copy only the
ojdbc8.jar
file. Thexstreams.jar
file is only required in environments that use the Oracle XStreams implementation. -
If you are using the XStreams implementation, 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/
The
LD_LIBRARY_PATH
environment variable is not required if you run the Oracle LogMiner implementation.
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.
{
"name": "inventory-connector", // <`>`
"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.
{
"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.
{
"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 |
{
"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.
-
The Debezium Oracle connector is installed.
-
Create a configuration for the Oracle connector.
-
Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.
When 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:
-
Database history connector configuration properties that control how Debezium processes events that it reads from the database history topic.
-
Pass-through database driver properties that control the behavior of the database driver.
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 |
|
|
The maximum number of tasks that should be created 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, and underscores. |
|
|
The adapter implementation that the connector uses when it streams database changes. You can set the following values:
|
|
initial |
Specifies the mode that the connector uses to take snapshots of a captured table. You can set the following values:
After the snapshot is complete, the connector continues to read change events from the database’s redo logs. |
|
No default |
Specifies the table rows to include in a snapshot. |
|
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 |
|
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 |
|
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 identifier is of the form |
|
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: |
|
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 are of the form |
|
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 names for columns are of the form |
|
No default |
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 pdbName.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. 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. |
|
|
Specifies how the connector should handle floating point values for
|
|
|
Specifies how the connector should react to exceptions during processing of events. You can set one of the following options:
|
|
|
A positive integer value that specifies the maximum size of the blocking queue.
Change events read from the database log are placed in the blocking queue before they are written to Kafka.
This queue can provide backpressure to the binlog reader when, for example, writes to Kafka are slow, or if Kafka is not available.
Events that appear in the queue are not included in the offsets that the connector records periodically.
Always specify a value that is larger than the maximum batch size that specified for the |
|
|
A positive integer value that specifies the maximum size of each batch of events to process during each iteration of this connector. |
|
|
Long value for the maximum size in bytes of the blocking queue. To activate the feature, set the value to a positive long data type. |
|
|
Positive integer value that specifies the number of milliseconds the connector should wait during each iteration for new change events to appear. |
|
|
Controls whether a delete event is followed by a tombstone event. The following values are possible:
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 regular expressions, separated by semi-colons, that match the fully-qualified tables and columns to map a primary 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: |
|
No default |
An optional comma-separated list of regular expressions for masking column names in change event messages by replacing characters with asterisks ( |
|
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 |
|
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 |
|
|
Specifies, in milliseconds, how frequently the connector sends messages to a heartbeat topic. |
|
|
Specifies the string that prefixes the name of the topic to which the connector sends heartbeat messages. |
|
No default |
Specifies an interval in milliseconds that the connector waits after it starts before it takes a snapshot. |
|
|
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. |
|
|
Specifies whether field names are normalized to comply with Avro naming requirements. For more information, see Avro naming. |
|
|
Set the property to See Transaction Metadata for additional details. |
|
|
The mining strategy controls how Oracle LogMiner builds and uses a given data dictionary for resolving table and column ids to names. |
|
|
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. |
|
|
The maximum SCN interval size that this connector uses when reading from redo/archive logs. |
|
|
The starting SCN interval size that the connector uses for reading data from redo/archive logs. |
|
|
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. |
|
|
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. |
|
|
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. |
|
|
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. |
|
|
The number of content records that the connector fetches from the LogMiner content view. |
|
|
The number of hours in the past from SYSDATE to mine archive logs.
When the default setting ( |
|
|
Positive integer value that specifies the number of hours to retain long running transactions between redo log switches.
When set to 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 |
A comma-separated list of Oracle Real Application Clusters (RAC) node host names or addresses. This field is required to enable use with Oracle RAC. |
Debezium 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.
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. |
||
|
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. |
|
|
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 |
|
|
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 |
|
Deprecated and scheduled for removal in a future release; use |
|
A Boolean value that specifies whether the connector should record all DDL statements
The safe default is |
|
A Boolean value that specifies whether the connector should record all DDL statements
The safe default is |
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 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.
-
snapshot metrics; for monitoring the connector when performing snapshots
-
streaming metrics; for monitoring the connector when processing change events
-
schema history metrics; for monitoring the status of the connector’s schema history
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=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
|
The last snapshot event that the connector has read. |
|
|
The number of milliseconds since the connector has read and processed the most recent event. |
|
|
The total number of events that this connector has seen since last started or reset. |
|
|
The number of events that have been filtered by include/exclude list filtering rules configured on the connector. |
|
|
The list of tables that are monitored by the connector. |
|
|
The length the queue used to pass events between the snapshotter and the main Kafka Connect loop. |
|
|
The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop. |
|
|
The total number of tables that are being included in the snapshot. |
|
|
The number of tables that the snapshot has yet to copy. |
|
|
Whether the snapshot was started. |
|
|
Whether the snapshot was aborted. |
|
|
Whether the snapshot completed. |
|
|
The total number of seconds that the snapshot has taken so far, even if not complete. |
|
|
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. |
|
|
The maximum buffer of the queue in bytes. It will be enabled if |
|
|
The current data of records in the queue in bytes. |
Streaming Metrics
The MBean is debezium.oracle:type=connector-metrics,context=streaming,server=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
|
The last streaming event that the connector has read. |
|
|
The number of milliseconds since the connector has read and processed the most recent event. |
|
|
The total number of events that this connector has seen since last started or reset. |
|
|
The number of events that have been filtered by include/exclude list filtering rules configured on the connector. |
|
|
The list of tables that are monitored by the connector. |
|
|
The length the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop. |
|
|
Flag that denotes whether the connector is currently connected to the database server. |
|
|
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. |
|
|
The number of processed transactions that were committed. |
|
|
The coordinates of the last received event. |
|
|
Transaction identifier of the last processed transaction. |
|
|
The maximum buffer of the queue in bytes. |
|
|
The current data of records in the queue in bytes. |
The Debezium Oracle connector also provides the following additional streaming metrics:
Attributes | Type | Description |
---|---|---|
|
The most recent system change number that has been processed. |
|
|
The oldest system change number in the transaction buffer. |
|
|
The last committed system change number from the transaction buffer. |
|
|
The system change number currently written to the connector’s offsets. |
|
|
Array of the log files that are currently mined. |
|
|
The minimum number of logs specified for any LogMiner session. |
|
|
The maximum number of logs specified for any LogMiner session. |
|
|
Array of the current state for each mined logfile with the format |
|
|
The number of times the database has performed a log switch for the last day. |
|
|
The number of DML operations observed in the last LogMiner session query. |
|
|
The maximum number of DML operations observed while processing a single LogMiner session query. |
|
|
The total number of DML operations observed. |
|
|
The total number of LogMiner session query (aka batches) performed. |
|
|
The duration of the last LogMiner session query’s fetch in milliseconds. |
|
|
The maximum duration of any LogMiner session query’s fetch in milliseconds. |
|
|
The duration for processing the last LogMiner query batch results in milliseconds. |
|
|
The time in milliseconds spent parsing DML event SQL statements. |
|
|
The duration in milliseconds to start the last LogMiner session. |
|
|
The longest duration in milliseconds to start a LogMiner session. |
|
|
The total duration in milliseconds spent by the connector starting LogMiner sessions. |
|
|
The minimum duration in milliseconds spent processing results from a single LogMiner session. |
|
|
The maximum duration in milliseconds spent processing results from a single LogMiner session. |
|
|
The total duration in milliseconds spent processing results from LogMiner sessions. |
|
|
The total duration in milliseconds spent by the JDBC driver fetching the next row to be processed from the log mining view. |
|
|
The total number of rows processed from the log mining view across all sessions. |
|
|
The number of entries fetched by the log mining query per database round-trip. |
|
|
The number of milliseconds the connector sleeps before fetching another batch of results from the log mining view. |
|
|
The maximum number of rows/second processed from the log mining view. |
|
|
The average number of rows/second processed from the log mining. |
|
|
The average number of rows/second processed from the log mining view for the last batch. |
|
|
The number of connection problems detected. |
|
|
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 |
|
|
The number of current active transactions in the transaction buffer. |
|
|
The number of committed transactions in the transaction buffer. |
|
|
The number of rolled back transactions in the transaction buffer. |
|
|
The average number of committed transactions per second in the transaction buffer. |
|
|
The number of registered DML operations in the transaction buffer. |
|
|
The time difference in milliseconds between when a change occurred in the transaction logs and when its added to the transaction buffer. |
|
|
The maximum time difference in milliseconds between when a change occurred in the transaction logs and when its added to the transaction buffer. |
|
|
The minimum time difference in milliseconds between when a change occurred in the transaction logs and when its added to the transaction buffer. |
|
|
An array of abandoned transaction identifiers removed from the transaction buffer due to their age.
See |
|
|
An array of transaction identifiers that have been mined and rolled back in the transaction buffer. |
|
|
The duration of the last transaction buffer commit operation in milliseconds. |
|
|
The duration of the longest transaction buffer commit operation in milliseconds. |
|
|
The number of errors detected. |
|
|
The number of warnings detected. |
|
|
The number of times the system change number has been checked for advancement and remains unchanged.
This is an indicator that long-running transaction(s) are ongoing and preventing the connector from flushing the latest processed system change number to the connector’s offsets.
Under optimal operations, this should always be or remain close to |
Schema History Metrics
The MBean is debezium.oracle:type=connector-metrics,context=schema-history,server=<database.server.name>
.
Attributes | Type | Description |
---|---|---|
|
One of |
|
|
The time in epoch seconds at what recovery has started. |
|
|
The number of changes that were read during recovery phase. |
|
|
the total number of schema changes applied during recovery and runtime. |
|
|
The number of milliseconds that elapsed since the last change was recovered from the history store. |
|
|
The number of milliseconds that elapsed since the last change was applied. |
|
|
The string representation of the last change recovered from the history store. |
|
|
The string representation of the last applied change. |
Surrogate schema evolution
Oracle connector does not yet support updates in table schemas parsing DDL ALTER TABLE
statements.
This DDL can be manually performed by using Debezium signalling to trigger the update of the database schema.
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.
Name | Description |
---|---|
|
The name of the Oracle database. |
|
The name of the schema where changes are applied. |
|
An array containing the requested schema updates. |
|
Type of the schema change, usually |
|
The fully-qualified name of the table |
|
The fully-qualified name of the table |
|
The character set name used for the table if different from database default |
|
Array with the name of columns composing the primary key |
|
Array with the column metadata |
|
The name of the column |
|
The JDBC type of the column as defined at JDBC API |
|
The name of the column type |
|
The full column type definition |
|
The column character set if different from the default |
|
The length/size constraint of the column |
|
The scale of numeric column |
|
The position of the column in the table starting with |
|
Boolean |
|
Boolean |
|
Boolean |
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.
Column | Value |
---|---|
id |
|
type |
|
data |
|
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.
-
To use the XStream API, you must have a license for the GoldenGate product. Installing GoldenGate is not required.
Preparing the Database
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.
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;
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):
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;
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"
}
}
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.
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.
LogMiner adapter does not capture changes made by SYS or SYSTEM
Oracle uses the SYS
and SYSTEM
accounts for lots of internal changes and therefore the connector automatically filters changes made by these users when fetching changes from LogMiner.
Please make sure to use a non-SYS
, non-SYSTEM
user account for changes to be emitted by the Debezium Oracle connector.