Debezium Connector for Oracle
- Overview
- How the Oracle connector works
- Data change events
- Data type mappings
- Custom converters
- Setting up Oracle
- Deployment
- Connector properties
- Monitoring
- Surrogate schema evolution
- OpenLogReplicator support
- How OpenLogReplicator works
- Preparing the database
- Creating connector users
- Configuring the OpenLogReplicator adapter
- Building OpenLogReplicator
- Obtaining the Oracle JDBC driver for OpenLogReplicator
- OpenLogReplicator configuration
- OpenLogReplicator connector properties
- OpenLogReplicator ROWID support
- OpenLogReplicator XML support
- XStreams support
- Frequently Asked Questions
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 , the XStream API, or OpenLogReplicator.
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.
If the time needed to complete the initial snapshot exceeds the |
During a table’s snapshot, it’s possible for Oracle to raise an ORA-01466 exception. This happens when a user modifies the schema of the table or adds, changes, or drops an index or related object associated with the table being snapshot. In the event this happens, the connector will stop and the initial snapshot will need to be taken from the beginning. To remediate the problem, you can configure the |
Default workflow that the Oracle connector uses to perform an initial snapshot
The following workflow lists the steps that Debezium takes to create a snapshot.
These steps describe the process for a snapshot when the snapshot.mode
configuration property is set to its default value, which is initial
.
You can customize the way that the connector creates snapshots by changing the value of the snapshot.mode
property.
If you configure a different snapshot mode, the connector completes the snapshot by using a modified version of this workflow.
When the snapshot mode is set to the default, the connector completes the following tasks to create a snapshot:
-
Establish a connection to the database.
-
Determine the tables to be captured. By default, the connector captures all tables except those with schemas that exclude them from capture. After the snapshot completes, the connector continues to stream data for the specified tables. If you want the connector to capture data only from specific tables you can direct the connector to capture the data for only a subset of tables or table elements by setting properties such as
table.include.list
ortable.exclude.list
. -
Obtain a
ROW SHARE MODE
lock on each of the captured tables to prevent structural changes from occurring during creation of the snapshot. Debezium holds the locks for only a short time. -
Read the current system change number (SCN) position from the server’s redo log.
-
Capture the structure of all database tables, or all tables that are designated for capture. The connector persists schema information in its internal database schema history topic. The schema history provides information about the structure that is in effect when a change event occurs.
By default, the connector captures the schema of every table in the database that is in capture mode, including tables that are not configured for capture. If tables are not configured for capture, the initial snapshot captures only their structure; it does not capture any table data. For more information about why snapshots persist schema information for tables that you did not include in the initial snapshot, see Understanding why initial snapshots capture the schema for all tables.
-
Release the locks obtained in Step 3. Other database clients can now write to any previously locked tables.
-
At the SCN position that was read in Step 4, the connector scans the tables that are designated for capture (
SELECT * FROM … AS OF SCN 123
). During the scan, the connector completes the following tasks:-
Confirms that the table was created before the snapshot began. If the table was created after the snapshot began, the connector skips the table. After the snapshot is complete, and the connector transitions to streaming, it emits change events for any tables that were created after the snapshot began.
-
Produces a
read
event for each row that is captured from a table. Allread
events contain the same SCN position, which is the SCN position that was obtained in step 4. -
Emits each
read
event to the Kafka topic for the source table. -
Releases data table locks, if applicable.
-
-
Record the successful completion of the snapshot in the connector offsets.
The resulting initial snapshot captures the current state of each row in the captured tables. From this baseline state, the connector captures subsequent changes as they occur.
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 |
---|---|
|
Perform snapshot on each connector start. After the snapshot completes, the connector begins to stream event records for subsequent database changes. |
|
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 performs a database snapshot and stops before streaming any change event records, not allowing any subsequent change events to be captured. |
|
Deprecated, see |
|
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 |
|
Deprecated, see |
|
Set this option to restore a database schema 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 schema history topic that experiences unexpected growth. |
|
After the connector starts, it performs a snapshot only if it detects one of the following circumstances:
|
|
Set the snapshot mode to |
|
The |
For more information, see snapshot.mode
in the table of connector configuration properties.
Understanding why initial snapshots capture the schema history for all tables
The initial snapshot that a connector runs captures two types of information:
- Table data
-
Information about
INSERT
,UPDATE
, andDELETE
operations in tables that are named in the connector’stable.include.list
property. - Schema data
-
DDL statements that describe the structural changes that are applied to tables. Schema data is persisted to both the internal schema history topic, and to the connector’s schema change topic, if one is configured.
After you run an initial snapshot, you might notice that the snapshot captures schema information for tables that are not designated for capture. By default, initial snapshots are designed to capture schema information for every table that is present in the database, not only from tables that are designated for capture. Connectors require that the table’s schema is present in the schema history topic before they can capture a table. By enabling the initial snapshot to capture schema data for tables that are not part of the original capture set, Debezium prepares the connector to readily capture event data from these tables should that later become necessary. If the initial snapshot does not capture a table’s schema, you must add the schema to the history topic before the connector can capture data from the table.
In some cases, you might want to limit schema capture in the initial snapshot. This can be useful when you want to reduce the time required to complete a snapshot. Or when Debezium connects to the database instance through a user account that has access to multiple logical databases, but you want the connector to capture changes only from tables in a specific logic database.
-
Capturing data from tables not captured by the initial snapshot (no schema change)
-
Capturing data from tables not captured by the initial snapshot (schema change)
-
Setting the
schema.history.internal.store.only.captured.tables.ddl
property to specify the tables from which to capture schema information. -
Setting the
schema.history.internal.store.only.captured.databases.ddl
property to specify the logical databases from which to capture schema changes.
Capturing data from tables not captured by the initial snapshot (no schema change)
In some cases, you might want the connector to capture data from a table whose schema was not captured by the initial snapshot. Depending on the connector configuration, the initial snapshot might capture the table schema only for specific tables in the database. If the table schema is not present in the history topic, the connector fails to capture the table, and reports a missing schema error.
You might still be able to capture data from the table, but you must perform additional steps to add the table schema.
-
You want to capture data from a table with a schema that the connector did not capture during the initial snapshot.
-
All entries for the table in the transaction log use the same schema. For information about capturing data from a new table that has undergone structural changes, see Capturing data from tables not captured by the initial snapshot (schema change).
-
Stop the connector.
-
Remove the internal database schema history topic that is specified by the
schema.history.internal.kafka.topic property
. -
In the connector configuration:
-
Set the
snapshot.mode
toschema_only_recovery
. -
(Optional) Set the value of
schema.history.internal.store.only.captured.tables.ddl
tofalse
to ensure that in the future the connector can readily capture data for tables that are not currently designated for capture. Connectors can capture data from a table only if the table’s schema history is present in the history topic. -
Add the tables that you want the connector to capture to
table.include.list
.
-
-
Restart the connector. The snapshot recovery process rebuilds the schema history based on the current structure of the tables.
-
(Optional) After the snapshot completes, initiate an incremental snapshot on the newly added tables. The incremental snapshot first streams the historical data of the newly added tables, and then resumes reading changes from the redo and archive logs for previously configured tables, including changes that occur while that connector was off-line.
-
(Optional) Reset the
snapshot.mode
back toschema_only
to prevent the connector from initiating recovery after a future restart.
Capturing data from tables not captured by the initial snapshot (schema change)
If a schema change is applied to a table, records that are committed before the schema change have different structures than those that were committed after the change. When Debezium captures data from a table, it reads the schema history to ensure that it applies the correct schema to each event. If the schema is not present in the schema history topic, the connector is unable to capture the table, and an error results.
If you want to capture data from a table that was not captured by the initial snapshot, and the schema of the table was modified, you must add the schema to the history topic, if it is not already available. You can add the schema by running a new schema snapshot, or by running an initial snapshot for the table.
-
You want to capture data from a table with a schema that the connector did not capture during the initial snapshot.
-
A schema change was applied to the table so that the records to be captured do not have a uniform structure.
- Initial snapshot captured the schema for all tables (
store.only.captured.tables.ddl
was set tofalse
) -
-
Edit the
table.include.list
property to specify the tables that you want to capture. -
Restart the connector.
-
Initiate an incremental snapshot if you want to capture existing data from the newly added tables.
-
- Initial snapshot did not capture the schema for all tables (
store.only.captured.tables.ddl
was set totrue
) -
If the initial snapshot did not save the schema of the table that you want to capture, complete one of the following procedures:
- Procedure 1: Schema snapshot, followed by incremental snapshot
-
In this procedure, the connector first performs a schema snapshot. You can then initiate an incremental snapshot to enable the connector to synchronize data.
-
Stop the connector.
-
Remove the internal database schema history topic that is specified by the
schema.history.internal.kafka.topic property
. -
Clear the offsets in the configured Kafka Connect
offset.storage.topic
. For more information about how to remove offsets, see the Debezium community FAQ.Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.
-
Set values for properties in the connector configuration as described in the following steps:
-
Set the value of the
snapshot.mode
property toschema_only
. -
Edit the
table.include.list
to add the tables that you want to capture.
-
-
Restart the connector.
-
Wait for Debezium to capture the schema of the new and existing tables. Data changes that occurred any tables after the connector stopped are not captured.
-
To ensure that no data is lost, initiate an incremental snapshot.
-
- Procedure 2: Initial snapshot, followed by optional incremental snapshot
-
In this procedure the connector performs a full initial snapshot of the database. As with any initial snapshot, in a database with many large tables, running an initial snapshot can be a time-consuming operation. After the snapshot completes, you can optionally trigger an incremental snapshot to capture any changes that occur while the connector is off-line.
-
Stop the connector.
-
Remove the internal database schema history topic that is specified by the
schema.history.internal.kafka.topic property
. -
Clear the offsets in the configured Kafka Connect
offset.storage.topic
. For more information about how to remove offsets, see the Debezium community FAQ.Removing offsets should be performed only by advanced users who have experience in manipulating internal Kafka Connect data. This operation is potentially destructive, and should be performed only as a last resort.
-
Edit the
table.include.list
to add the tables that you want to capture. -
Set values for properties in the connector configuration as described in the following steps:
-
Set the value of the
snapshot.mode
property toinitial
. -
(Optional) Set
schema.history.internal.store.only.captured.tables.ddl
tofalse
.
-
-
Restart the connector. The connector takes a full database snapshot. After the snapshot completes, the connector transitions to streaming.
-
(Optional) To capture any data that changed while the connector was off-line, initiate an incremental snapshot.
-
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. You might want to perform an ad hoc snapshot after any of the following changes occur in your Debezium environment:
-
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. Also, the snapshot can capture a subset of the contents of the table(s) 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
or blocking
, and provide the names of the tables to include in the snapshot, as described in the following table:
Field | Default | Value |
---|---|---|
|
|
Specifies the type of snapshot that you want to run. |
|
N/A |
An array that contains regular expressions matching the fully-qualified names of the tables to include in the snapshot. |
|
N/A |
An optional array that specifies a set of additional conditions that the connector evaluates to determine the subset of records to include in a snapshot.
|
|
N/A |
An optional string that specifies the column name that the connector uses as the primary key of a table during the snapshot process. |
You initiate an ad hoc incremental snapshot by adding an entry with the execute-snapshot
signal type to the signaling table, or by sending a signal message to a Kafka signaling topic.
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.
For more information, see Incremental snapshots.
You initiate an ad hoc blocking snapshot by adding an entry with the execute-snapshot
signal type to the signaling table or signaling topic.
After the connector processes the message, it begins the snapshot operation.
The connector temporarily stops streaming, and then initiates a snapshot of the specified table, following the same process that it uses during an initial snapshot.
After the snapshot completes, the connector resumes streaming.
For more information, see Blocking 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 1024 rows.
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.
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.
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.
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 window 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.
Currently, you can use either of the following methods to initiate an incremental snapshot:
The Debezium connector for Oracle does not support schema changes while an incremental snapshot is running. |
Triggering an incremental snapshot
To initiate an incremental snapshot, you can send an ad hoc snapshot signal to the signaling table on the source database.
You submit snapshot signals 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 type of snapshot operation.
Debezium currently supports the incremental
and blocking
snapshot types.
To specify the tables to include in the snapshot, provide a data-collections
array that lists the tables, or an array of regular expressions used to match 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 interprets the empty array to mean that no action is required, and it does not perform a snapshot.
If the name of a table that you want to include in a snapshot contains a dot ( |
-
-
A signaling data collection exists on the source database.
-
The signaling data collection is specified in the
signal.data.collection
property.
-
-
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": ["<fullyQualfiedTableName>","<fullyQualfiedTableName>"],"type":"<snapshotType>","additional-conditions":[{"data-collection": "<fullyQualfiedTableName>", "filter": "<additional-condition>"}]}');
For example,
INSERT INTO db1.myschema.debezium_signal (id, type, data) (1) values ('ad-hoc-1', (2) 'execute-snapshot', (3) '{"data-collections": ["db1.schema1.table1", "db1.schema1.table2"], (4) "type":"incremental", (5) "additional-conditions":[{"data-collection": "db1.schema1.table1" ,"filter":"color=\'blue\'"}]}'); (6)
The values of the
id
,type
, anddata
parameters in the command correspond to the fields of the signaling table.
The following table describes the parameters in the example:Table 3. Descriptions of fields in a SQL command for sending an incremental snapshot signal to the signaling table Item Value Description 1
database.schema.debezium_signal
Specifies the fully-qualified name of the signaling table on the source database.
2
ad-hoc-1
The
id
parameter specifies an arbitrary string that is assigned as theid
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 ownid
string as a watermarking signal.3
execute-snapshot
The
type
parameter specifies the operation that the signal is intended to trigger.4
data-collections
A required component of the
data
field of a signal that specifies an array of table names or regular expressions to match table names to include in the snapshot.
The array lists regular expressions that use the formatdatabase.schema.table
to match the fully-qualified names of the tables. This format is the same as the one that you use to specify the name of the connector’s signaling table.5
incremental
An optional
type
component of thedata
field of a signal that specifies the type of snapshot operation to run.
Valid values areincremental
andblocking
.
If you do not specify a value, the connector defaults to performing an incremental snapshot.6
additional-conditions
An optional array that specifies a set of additional conditions that the connector evaluates to determine the subset of records to include in a snapshot.
Each additional condition is an object withdata-collection
andfilter
properties. You can specify different filters for each data collection.
* Thedata-collection
property is the fully-qualified name of the data collection that the filter applies to. For more information about theadditional-conditions
parameter, see Running an ad hoc incremental snapshots withadditional-conditions
.
Running an ad hoc incremental snapshots with additional-conditions
If you want a snapshot to include only a subset of the content in a table, you can modify the signal request by appending an additional-conditions
parameter to the snapshot signal.
The SQL query for a typical snapshot takes the following form:
SELECT * FROM <tableName> ....
By adding an additional-conditions
parameter, you append a WHERE
condition to the SQL query, as in the following example:
SELECT * FROM <data-collection> WHERE <filter> ....
The following example shows a SQL query to send an ad hoc incremental snapshot request with an additional condition to the signaling table:
INSERT INTO <signalTable> (id, type, data) VALUES ('<id>', '<snapshotType>', '{"data-collections": ["<fullyQualfiedTableName>","<fullyQualfiedTableName>"],"type":"<snapshotType>","additional-conditions":[{"data-collection": "<fullyQualfiedTableName>", "filter": "<additional-condition>"}]}');
For example, suppose you have a products
table that contains the following columns:
-
id
(primary key) -
color
-
quantity
If you want an incremental snapshot of the products
table to include only the data items where color=blue
, you can use the following SQL statement to trigger the snapshot:
INSERT INTO db1.myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["db1.schema1.products"],"type":"incremental", "additional-conditions":[{"data-collection": "db1.schema1.products", "filter": "color=blue"}]}');
The additional-conditions
parameter also enables you to pass conditions that are based on more than one column.
For example, using the products
table from the previous example, you can submit a query that triggers an incremental snapshot that includes the data of only those items for which color=blue
and quantity>10
:
INSERT INTO db1.myschema.debezium_signal (id, type, data) VALUES('ad-hoc-1', 'execute-snapshot', '{"data-collections": ["db1.schema1.products"],"type":"incremental", "additional-conditions":[{"data-collection": "db1.schema1.products", "filter": "color=blue AND quantity>10"}]}');
The following example, shows the JSON for an incremental snapshot event that is captured by a connector.
{
"before":null,
"after": {
"pk":"1",
"value":"New data"
},
"source": {
...
"snapshot":"incremental" (1)
},
"op":"r", (2)
"ts_ms":"1620393591654",
"ts_us":"1620393591654547",
"ts_ns":"1620393591654547920",
"transaction":null
}
Item | Field name | Description |
---|---|---|
1 |
|
Specifies the type of snapshot operation to run. |
2 |
|
Specifies the event type. |
Using the Kafka signaling channel to trigger an incremental snapshot
You can send a message to the configured Kafka topic to request the connector to run an ad hoc incremental snapshot.
The key of the Kafka message must match the value of the topic.prefix
connector configuration option.
The value of the message is a JSON object with type
and data
fields.
The signal type is execute-snapshot
, and the data
field must have the following fields:
Field | Default | Value |
---|---|---|
|
|
The type of the snapshot to be executed.
Currently Debezium supports the |
|
N/A |
An array of comma-separated regular expressions that match the fully-qualified names of tables to include in the snapshot. |
|
N/A |
An optional array of additional conditions that specifies criteria that the connector evaluates to designate a subset of records to include in a snapshot. |
execute-snapshot
Kafka messageKey = `test_connector` Value = `{"type":"execute-snapshot","data": {"data-collections": ["{collection-container}.table1", "{collection-container}.table2"], "type": "INCREMENTAL"}}`
Debezium uses the additional-conditions
field to select a subset of a table’s content.
Typically, when Debezium runs a snapshot, it runs a SQL query such as:
SELECT * FROM <tableName> ….
When the snapshot request includes an additional-conditions
property, the data-collection
and filter
parameters of the property are appended to the SQL query, for example:
SELECT * FROM <data-collection> WHERE <filter> ….
For example, given a products
table with the columns id
(primary key), color
, and brand
, if you want a snapshot to include only content for which color='blue'
, when you request the snapshot, you could add the additional-conditions
property to filter the content:
:leveloffset: +1
Key = `test_connector`
Value = `{"type":"execute-snapshot","data": {"data-collections": ["db1.schema1.products"], "type": "INCREMENTAL", "additional-conditions": [{"data-collection": "db1.schema1.products" ,"filter":"color='blue'"}]}}`
You can also use the additional-conditions
property to pass conditions based on multiple columns.
For example, using the same products
table as in the previous example, if you want a snapshot to include only the content from the products
table for which color='blue'
, and brand='MyBrand'
, you could send the following request:
:leveloffset: +1
Key = `test_connector`
Value = `{"type":"execute-snapshot","data": {"data-collections": ["db1.schema1.products"], "type": "INCREMENTAL", "additional-conditions": [{"data-collection": "db1.schema1.products" ,"filter":"color='blue' AND brand='MyBrand'"}]}}`
Stopping an incremental snapshot
In some situations, it might be necessary to stop an incremental snapshot. For example, you might realize that snapshot was not configured correctly, or maybe you want to ensure that resources are available for other database operations. You can stop a snapshot that is already running by sending a signal to the signaling table on the source database.
You submit a stop snapshot signal to the signaling table by sending it in a SQL INSERT
query.
The stop-snapshot signal specifies the type
of the snapshot operation as incremental
, and optionally specifies the tables that you want to omit from the currently running snapshot.
After Debezium detects the change in the signaling table, it reads the signal, and stops the incremental snapshot operation if it’s in progress.
You can also stop an incremental snapshot by sending a JSON message to the Kafka signaling topic.
-
-
A signaling data collection exists on the source database.
-
The signaling data collection is specified in the
signal.data.collection
property.
-
-
Send a SQL query to stop the ad hoc incremental snapshot to the signaling table:
INSERT INTO <signalTable> (id, type, data) values ('<id>', 'stop-snapshot', '{"data-collections": ["<fullyQualfiedTableName>","<fullyQualfiedTableName>"],"type":"incremental"}');
For example,
INSERT INTO db1.myschema.debezium_signal (id, type, data) (1) values ('ad-hoc-1', (2) 'stop-snapshot', (3) '{"data-collections": ["db1.schema1.table1", "db1.schema1.table2"], (4) "type":"incremental"}'); (5)
The values of the
id
,type
, anddata
parameters in the signal command correspond to the fields of the signaling table.
The following table describes the parameters in the example:Table 6. Descriptions of fields in a SQL command for sending a stop incremental snapshot signal to the signaling table Item Value Description 1
database.schema.debezium_signal
Specifies the fully-qualified name of the signaling table on the source database.
2
ad-hoc-1
The
id
parameter specifies an arbitrary string that is assigned as theid
identifier for the signal request.
Use this string to identify logging messages to entries in the signaling table. Debezium does not use this string.3
stop-snapshot
Specifies
type
parameter specifies the operation that the signal is intended to trigger.4
data-collections
An optional component of the
data
field of a signal that specifies an array of table names or regular expressions to match table names to remove from the snapshot.
The array lists regular expressions which match tables by their fully-qualified names in the formatdatabase.schema.table
If you omit this component from the
data
field, the signal stops the entire incremental snapshot that is in progress.5
incremental
A required component of the
data
field of a signal that specifies the type of snapshot operation that is to be stopped.
Currently, the only valid option isincremental
.
If you do not specify atype
value, the signal fails to stop the incremental snapshot.
Using the Kafka signaling channel to stop an incremental snapshot
You can send a signal message to the configured Kafka signaling topic to stop an ad hoc incremental snapshot.
The key of the Kafka message must match the value of the topic.prefix
connector configuration option.
The value of the message is a JSON object with type
and data
fields.
The signal type is stop-snapshot
, and the data
field must have the following fields:
Field | Default | Value |
---|---|---|
|
|
The type of the snapshot to be executed.
Currently Debezium supports only the |
|
N/A |
An optional array of comma-separated regular expressions that match the fully-qualified names of the tables an array of table names or regular expressions to match table names to remove from the snapshot. |
The following example shows a typical stop-snapshot
Kafka message:
:leveloffset: +1
Key = `test_connector`
Value = `{"type":"stop-snapshot","data": {"data-collections": ["db1.schema1.table1", "db1.schema1.table2"], "type": "INCREMENTAL"}}`
Custom snapshotter SPI
For more advanced uses, you can fine-tune control of the snapshot by implementing one of the following interfaces:
io.debezium.snapshot.spi.Snapshotter
-
Controls whether the connector takes a snapshot.
io.debezium.snapshot.spi.SnapshotQuery
-
Controls how data is queried during a snapshot.
io.debezium.snapshot.spi.SnapshotLock
-
Controls whether the connector locks tables when taking a snapshot.
/**
* {@link Snapshotter} is used to determine the following details about the snapshot process:
* <p>
* - Whether a snapshot occurs. <br>
* - Whether streaming continues during the snapshot. <br>
* - Whether the snapshot includes schema (if supported). <br>
* - Whether to snapshot data or schema following an error.
* <p>
* Although Debezium provides many default snapshot modes,
* to provide more advanced functionality, such as partial snapshots,
* you can customize implementation of the interface.
* For more information, see the documentation.
*
*
*
*/
@Incubating
public interface Snapshotter extends Configurable {
/**
* @return the name of the snapshotter.
*
*
*/
String name();
/**
* @param offsetExists is {@code true} when the connector has an offset context (i.e. restarted)
* @param snapshotInProgress is {@code true} when the connector is started, but a snapshot is already in progress
*
* @return {@code true} if the snapshotter should take a data snapshot
*/
boolean shouldSnapshotData(boolean offsetExists, boolean snapshotInProgress);
/**
* @param offsetExists is {@code true} when the connector has an offset context (i.e. restarted)
* @param snapshotInProgress is {@code true} when the connector is started, but a snapshot is already in progress
*
* @return {@code true} if the snapshotter should take a schema snapshot
*/
boolean shouldSnapshotSchema(boolean offsetExists, boolean snapshotInProgress);
/**
* @return {@code true} if the snapshotter should stream after taking a snapshot
*/
boolean shouldStream();
/**
* @return {@code true} whether the schema can be recovered if database schema history is corrupted.
*/
boolean shouldSnapshotOnSchemaError();
/**
* @return {@code true} whether the snapshot should be re-executed when there is a gap in data stream.
*/
boolean shouldSnapshotOnDataError();
/**
*
* @return {@code true} if streaming should resume from the start of the snapshot
* transaction, or {@code false} for when a connector resumes and takes a snapshot,
* streaming should resume from where streaming previously left off.
*/
default boolean shouldStreamEventsStartingFromSnapshot() {
return true;
}
/**
* Lifecycle hook called after the snapshot phase is successful.
*/
default void snapshotCompleted() {
// no operation
}
/**
* Lifecycle hook called after the snapshot phase is aborted.
*/
default void snapshotAborted() {
// no operation
}
}
/**
* {@link SnapshotQuery} is used to determine the query used during a data snapshot
*
*
*/
public interface SnapshotQuery extends Configurable, Service {
/**
* @return the name of the snapshot lock.
*
*
*/
String name();
/**
* Generate a valid query string for the specified table, or an empty {@link Optional}
* to skip snapshotting this table (but that table will still be streamed from)
*
* @param tableId the table to generate a query for
* @param snapshotSelectColumns the columns to be used in the snapshot select based on the column
* include/exclude filters
* @return a valid query string, or none to skip snapshotting this table
*/
Optional<String> snapshotQuery(String tableId, List<String> snapshotSelectColumns);
}
/**
* {@link SnapshotLock} is used to determine the table lock mode used during schema snapshot
*
*
*/
public interface SnapshotLock extends Configurable, Service {
/**
* @return the name of the snapshot lock.
*
*
*/
String name();
/**
* Returns a SQL statement for locking the given table during snapshotting, if required by the specific snapshotter
* implementation.
*/
Optional<String> tableLockingStatement(Duration lockTimeout, String tableId);
}
Blocking snapshots
To provide more flexibility in managing snapshots, Debezium includes a supplementary ad hoc snapshot mechanism, known as a blocking snapshot. Blocking snapshots rely on the Debezium mechanism for sending signals to a Debezium connector.
A blocking snapshot behaves just like an initial snapshot, except that you can trigger it at run time.
You might want to run a blocking snapshot rather than use the standard initial snapshot process in the following situations:
-
You add a new table and you want to complete the snapshot while the connector is running.
-
You add a large table, and you want the snapshot to complete in less time than is possible with an incremental snapshot.
When you run a blocking snapshot, Debezium stops streaming, and then initiates a snapshot of the specified table, following the same process that it uses during an initial snapshot. After the snapshot completes, the streaming is resumed.
You can set the following properties in the data
component of a signal:
-
data-collections: to specify which tables must be snapshot.
-
data-collections: Specifies the tables that you want the snapshot to include.
This property accepts a comma-separated list of regular expressions that match fully-qualified table names. The behavior of the property is similar to the behavior of thetable.include.list
property, which specifies the tables to capture in a blocking snapshot. -
additional-conditions: You can specify different filters for different table.
-
The
data-collection
property is the fully-qualified name of the table for which the filter will be applied, and can be case-sensitive or case-insensitive depending on the database. -
The
filter
property will have the same value used in thesnapshot.select.statement.overrides
, the fully-qualified name of the table that should match by case.
-
For example:
{"type": "blocking", "data-collections": ["schema1.table1", "schema1.table2"], "additional-conditions": [{"data-collection": "schema1.table1", "filter": "SELECT * FROM [schema1].[table1] WHERE column1 = 0 ORDER BY column2 DESC"}, {"data-collection": "schema1.table2", "filter": "SELECT * FROM [schema1].[table2] WHERE column2 > 0"}]}
A delay might exist between the time that you send the signal to trigger the snapshot, and the time when streaming stops and the snapshot starts. As a result of this delay, after the snapshot completes, the connector might emit some event records that duplicate records captured by the snapshot.
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:
topicPrefix.schemaName.tableName
The following list provides definitions for the components of the default name:
- topicPrefix
-
The topic prefix as specified by the
topic.prefix
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 schema 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 history topic
When a database client queries a database, the client uses the database’s current schema. However, the database schema can be changed at any time, which means that the connector must be able to identify what the schema was at the time each insert, update, or delete operation was recorded. Also, a connector cannot necessarily apply the current schema to every event. If an event is relatively old, it’s possible that it was recorded before the current schema was applied.
To ensure correct processing of events that occur after a schema change, Oracle includes in the redo log not only the row-level changes that affect the data, but also the DDL statements that are applied to the database. As the connector encounters these DDL statements in the redo log, it parses them and updates an in-memory representation of each table’s schema. The connector uses this schema representation to identify the structure of the tables at the time of each insert, update, or delete operation and to produce the appropriate change event. In a separate database schema history Kafka topic, the connector records all DDL statements along with the position in the redo log where each DDL statement appeared.
When the connector restarts after either a crash or a graceful stop, it starts reading the redo log from a specific position, that is, from a specific point in time. The connector rebuilds the table structures that existed at this point in time by reading the database schema history Kafka topic and parsing all DDL statements up to the point in the redo log where the connector is starting.
This database schema history topic is internal for internal connector use only. Optionally, the connector can also emit schema change events to a different topic that is intended for consumer applications.
-
Default names for topics that receive Debezium event records.
Schema change topic
You can configure a Debezium Oracle connector to produce schema change events that describe structural changes that are applied to tables in the database.
The connector writes schema change events to a Kafka topic named <serverName>
, where serverName
is the namespace that is specified in the topic.prefix
configuration property.
Debezium emits a new message to the schema change 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). |
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 schema for the schema change event has the following elements:
name
-
The name of the schema change event message.
type
-
The type of the change event message.
version
-
The version of the schema. The version is an integer that is incremented each time the schema is changed.
fields
-
The fields that are included in the change event message.
The following example shows a typical schema in JSON format.
{
"schema": {
"type": "struct",
"fields": [
{
"type": "string",
"optional": false,
"field": "databaseName"
}
],
"optional": false,
"name": "io.debezium.connector.oracle.SchemaChangeKey",
"version": 1
},
"payload": {
"databaseName": "inventory"
}
}
The payload of a schema change event message includes the following elements:
ddl
-
Provides the SQL
CREATE
,ALTER
, orDROP
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.
By default, the connector uses the You can modify settings so that the schema history topic stores a different subset of tables. Use one of the following methods to alter the set of tables that the topic stores:
|
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 schema history topic. The internal database schema 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 schema history topic. For the database schema 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:
|
The schema change topic message format is in an incubating state and might change without notice. |
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": "3.0.2.Final",
"connector": "oracle",
"name": "server1",
"ts_ms": 1588252618953,
"ts_us": 1588252618953000,
"ts_ns": 1588252618953000000,
"snapshot": "true",
"db": "ORCLPDB1",
"schema": "DEBEZIUM",
"table": "CUSTOMERS",
"txId" : null,
"scn" : "1513734",
"commit_scn": "1513754",
"lcr_position" : null,
"rs_id": "001234.00012345.0124",
"ssn": 1,
"redo_thread": 1,
"user_name": "user",
"row_id": "AAASgjAAMAAAACnAAA"
},
"ts_ms": 1588252618953, (1)
"ts_us": 1588252618953987, (1)
"ts_ns": 1588252618953987512, (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
}
],
"attributes": [ (10)
{
"customAttribute": "attributeValue"
}
]
}
}
]
}
}
Item | Field name | Description |
---|---|---|
1 |
|
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 |
|
Identifies the database and the schema that contains the change. |
3 |
|
This field contains the DDL that is responsible for the schema change. |
4 |
|
An array of one or more items that contain the schema changes generated by a DDL command. |
5 |
|
Describes the kind of change. The
|
6 |
|
Full identifier of the table that was created, altered, or dropped.
In the case of a table rename, this identifier is a concatenation of |
7 |
|
Represents table metadata after the applied change. |
8 |
|
List of columns that compose the table’s primary key. |
9 |
|
Metadata for each column in the changed table. |
10 |
|
Custom attribute metadata for each table change. |
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",
"version": 1
},
"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
orEND
. id
-
String representation of the unique transaction identifier.
ts_ms
-
The time of a transaction boundary event (
BEGIN
orEND
event) at the data source. If the data source does not provide Debezium with the event time, then the field instead represents the time at which Debezium processes the event. 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
elements that indicates the number of events that the connector emits for changes that originate from a data collection.
The following example shows a typical transaction boundary message:
{
"status": "BEGIN",
"id": "5.6.641",
"ts_ms": 1486500577125,
"event_count": null,
"data_collections": null
}
{
"status": "END",
"id": "5.6.641",
"ts_ms": 1486500577691,
"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 topic.transaction
option,
the connector emits transaction events to the <topic.prefix>
.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",
"ts_us": "1580390884335741",
"ts_ns": "1580390884335741963",
"transaction": {
"id": "5.6.641",
"total_order": "1",
"data_collection_order": "1"
}
}
LogMiner Mining Strategies
Entries in the Oracle redo logs do not store the original SQL statements that users submit to make DML changes. Instead, a redo entry holds a set of change vectors and a set of object identifiers that represent the tablespace, table, and columns related to these vectors. In other words, redo log entries don’t include the names of the schemas, tables, or columns affected by DML changes.
The Debezium Oracle connector uses the log.mining.strategy
configuration property to control how Oracle LogMiner handles the lookup of the object identifiers in the change vectors.
In certain situations, one log mining strategy might prove more reliable than another with regard to schema changes.
However, before you choose a log mining strategy, it’s important to consider the implications it might have on performance and overhead.
Writing the data dictionary to redo logs
The default mining strategy is called redo_log_catalog
.
In this strategy, the database flushes a copy of the data dictionary to the redo logs immediately after each redo log switch.
This is the most reliable strategy for tracking schema changes that are interwoven with data changes, because Oracle LogMiner has a way to interpolate between the starting and ending data dictionary states across a series of change vectors.
However, the redo_log_catalog
mode is also the most expensive, because it requires several key steps to function.
First, this mode requires the data dictionary to be flushed to the redo logs after every log switch.
Flushing the logs after each switch can quickly consume valuable space in the archive log, and the high volume of archive logs might exceed the number that database administrators prepared for.
If you intend to use this mode, coordinate with your database administrators to ensure that the database is configured appropriately.
If you configure the connector to use the |
Using the online catalog directly
The next strategy mode, online_catalog
, works differently from the redo_log_catalog
mode.
When the strategy is set to online_catalog
, the database never flushes the data dictionary to the redo logs.
Instead, Oracle LogMiner always uses the most current data dictionary state to perform comparisons.
By always using the current dictionary, and eliminating flushing to the redo logs, this strategy requires less overhead, and operates more efficiently.
However, these benefits are offset by the inability to parse interwoven schema changes and data changes.
As a result, this strategy can sometimes result in event failures.
If LogMiner was unable to reconstruct the SQL reliability after a schema change, check the redo logs for evidence.
Look for references to tables with names like OBJ# 123456
(where the number is the table’s object identifier), or for columns with names like COL1
or COL2
.
When you configure the connector to use the online_catalog
strategy, take steps to ensure that the table schema and its indices remain static and free from change.
If the Debezium connector is configured to use the online_catalog
mode, and you must apply a schema change, perform the following steps:
-
Wait for the connector to capture all existing data changes (DML).
-
Perform the schema (DDL) change, and then wait for the connector to capture the change.
-
Resume data changes (DML) on the table.
Following this procedure helps to ensure that Oracle LogMiner can safely reconstruct the SQL for all data changes.
Hybrid approach
This is a new, experimental strategy that can be enabled by setting the strategy to hybrid
.
The goal of this strategy is to provide the reliability of the redo_log_catalog
strategy with the performance and low overhead of the online_catalog
strategy, without incurring the disadvantages of either strategy.
The hybrid
strategy works by primarily operating in the online_catalog
mode, meaning that the Debezium Oracle connector first delegates event reconstruction to Oracle LogMiner.
If Oracle LogMiner successfully reconstructs the SQL, Debezium processes the event normally, as if it were configured to use the online_catalog
strategy.
If the connector detects that Oracle LogMiner could not reconstruct the SQL, the connector attempts to reconstruct the SQL directly by using the schema history for that table object.
The connector reports a failure only if both Oracle LogMiner and the connector are unable to reconstruct the SQL.
You cannot use the |
Query Modes
The Debezium Oracle connector integrates with Oracle LogMiner by default.
This integration requires a specialized set of steps which includes generating a complex JDBC SQL query to ingest the changes recorded in the transaction logs as change events.
The V$LOGMNR_CONTENTS
view used by the JDBC SQL query does not have any indices to improve the query’s performance, and so there are different query modes that can be used that control how the SQL query is generated as a way to improve the query’s execution.
The log.mining.query.filter.mode
connector property can be configured with one of the following to influence how the JDBC SQL query is generated:
none
-
(Default) This mode creates a JDBC query that only filters based on the different operation types, such as inserts, updates, or deletes, at the database level. When filtering the data based on the schema, table, or username include/exclude lists, this is done during the processing loop within the connector.
This mode is often useful when capturing a small number of tables from a database that is not heavily saturated with changes. The generated query is quite simple, and focuses primarily on reading as quickly as possible with low database overhead. in
-
This mode creates a JDBC query that filters not only operation types at the database level, but also schema, table, and username include/exclude lists. The query’s predicates are generated using a SQL in-clause based on the values specified in the include/exclude list configuration properties.
This mode is often useful when capturing a large number of tables from a database that is heavily saturated with changes. The generated query is much more complex than thenone
mode, and focuses on reducing network overhead and performing as much filtering at the database level as possible.
Finally, do not specify regular expressions as part of schema and table include/exclude configuration properties. Using regular expressions will cause the connector to not match changes based on these configuration properties, causing changes to be missed. regex
-
This mode creates a JDBC query that filters not only operation types at the database level, but also schema, table, and username include/exclude lists. However, unlike the
in
mode, this mode generates a SQL query using the OracleREGEXP_LIKE
operator using a conjunction or disjunction depending on whether include or excluded values are specified.
This mode is often useful when capturing a variable number of tables that can be identified using a small number of regular expressions. The generated query is much more complex than any other mode, and focuses on reducing network overhead and performing as much filtering at the database level as possible.
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 read-only="false" preload="true" shared="false">
<data path="./data"/>
<index path="./index"/>
</file-store>
</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.
The Infinispan buffer implementation utilizes multiple cache configurations with different names.
There should be a cache defined for |
When supplying XML configuration as a JSON connector property value, line breaks must be omitted or replaced with a |
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 read-only="false" preload="true" shared="false">
<data path="./data"/>
<index path="./index"/>
</file-store>
</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 |
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 <topic.prefix>
.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.
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), orr
(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 theserver1
connector uses for all rows in theinventory.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 thebefore
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).
-
Username who made the change
-
The ROWID associated with the row
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.The
user_name
field is only populated 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.
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": "int64",
"optional": true,
"field": "ts_us"
},
{
"type": "int64",
"optional": true,
"field": "ts_ns"
},
{
"type": "string",
"optional": true,
"field": "txId"
},
{
"type": "string",
"optional": true,
"field": "scn"
},
{
"type": "string",
"optional": true,
"field": "commit_scn"
},
{
"type": "string",
"optional": true,
"field": "rs_id"
},
{
"type": "int64",
"optional": true,
"field": "ssn"
},
{
"type": "int32",
"optional": true,
"field": "redo_thread"
},
{
"type": "string",
"optional": true,
"field": "user_name"
},
{
"type": "boolean",
"optional": true,
"field": "snapshot"
},
{
"type": "string",
"optional": true,
"field": "row_id"
}
],
"optional": false,
"name": "io.debezium.connector.oracle.Source",
"field": "source"
},
{
"type": "string",
"optional": false,
"field": "op"
},
{
"type": "int64",
"optional": true,
"field": "ts_ms"
},
{
"type": "int64",
"optional": true,
"field": "ts_us"
},
{
"type": "int64",
"optional": true,
"field": "ts_ns"
}
],
"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": "3.0.2.Final",
"name": "server1",
"ts_ms": 1520085154000,
"ts_us": 1520085154000000,
"ts_ns": 1520085154000000000,
"txId": "6.28.807",
"scn": "2122185",
"commit_scn": "2122185",
"rs_id": "001234.00012345.0124",
"ssn": 1,
"redo_thread": 1,
"user_name": "user",
"snapshot": false,
"row_id": "AAASgjAAMAAAACnAAA"
},
"op": "c",
"ts_ms": 1532592105975,
"ts_us": 1532592105975741,
"ts_ns": 1532592105975741582
}
}
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
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 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": "3.0.2.Final",
"name": "server1",
"ts_ms": 1520085811000,
"ts_us": 1520085811000000,
"ts_ns": 1520085811000000000,
"txId": "6.9.809",
"scn": "2125544",
"commit_scn": "2125544",
"rs_id": "001234.00012345.0124",
"ssn": 1,
"redo_thread": 1,
"user_name": "user",
"snapshot": false,
"row_id": "AAASgjAAMAAAACnAAA"
},
"op": "u",
"ts_ms": 1532592713485,
"ts_us": 1532592713485152,
"ts_ns": 1532592713485152954,
}
}
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 isu
, 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 theupdate
database commit. -
The
after
field shows the updated state of the row, with theEMAIL
value now set toanne@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:
|
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": "3.0.2.Final",
"name": "server1",
"ts_ms": 1520085153000,
"ts_us": 1520085153000000,
"ts_ns": 1520085153000000000,
"txId": "6.28.807",
"scn": "2122184",
"commit_scn": "2122184",
"rs_id": "001234.00012345.0124",
"ssn": 1,
"redo_thread": 1,
"user_name": "user",
"snapshot": false,
"row_id": "AAASgjAAMAAAACnAAA"
},
"op": "d",
"ts_ms": 1532592105960,
"ts_us": 1532592105960854,
"ts_ns": 1532592105960854693
}
}
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 isd
, 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 isnull
, 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 thets_ms
,scn
, andtxId
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": "3.0.2.Final",
"connector": "oracle",
"name": "oracle_server",
"ts_ms": 1638974535000,
"ts_us": 1638974535000000,
"ts_ns": 1638974535000000000,
"snapshot": "false",
"db": "ORCLPDB1",
"sequence": null,
"schema": "DEBEZIUM",
"table": "TEST_TABLE",
"txId": "02000a0037030000",
"scn": "13234397",
"commit_scn": "13271102",
"lcr_position": null,
"rs_id": "001234.00012345.0124",
"ssn": 1,
"redo_thread": 1,
"user_name": "user"
},
"op": "t", (2)
"ts_ms": 1638974558961, (3)
"ts_us": 1638974558961987, (3)
"ts_ns": 1638974558961987251, (3)
"transaction": null
}
}
Item | Field name | Description |
---|---|---|
1 |
|
Mandatory field that describes the source metadata for the event. In a truncate event value, the
|
2 |
|
Mandatory string that describes the type of operation. The |
3 |
|
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 |
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
, andSTRUCT
. - 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.
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
Support for |
The following table describes how the connector maps binary and character large object (LOB) data types.
Oracle data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
n/a |
This data type is not supported |
|
|
Depending on the setting of the
|
|
|
n/a |
|
n/a |
This data type is not supported. |
|
n/a |
This data type is not supported. |
|
|
n/a |
|
n/a |
Depending on the setting of the
|
Oracle only supplies column values for If the value of a |
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 |
Oracle data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
n/a |
|
|
n/a |
|
|
When the When the |
|
|
|
|
|
|
|
|
|
|
|
When the When the |
|
|
When the When the |
|
|
|
|
|
When the When the |
|
|
|
|
|
When the When the |
As mention above, Oracle allows negative scales in NUMBER
type.
This can cause an issue during conversion to the Avro format when the number is represented as the Decimal
.
Decimal
type includes scale information, but Avro specification allows only positive values for the scale.
Depending on the schema registry used, it may result into Avro serialization failure.
To avoid this issue, you can use NumberToZeroScaleConverter
, which converts sufficiently high numbers (P - S >= 19) with negative scale into Decimal
type with zero scale.
It can be configured as follows:
converters=zero_scale
zero_scale.type=io.debezium.connector.oracle.converters.NumberToZeroScaleConverter
zero_scale.decimal.mode=precise
By default, the number is converted to Decimal
type (zero_scale.decimal.mode=precise
), but for completeness remaining two supported types (double
and string
) are supported as well.
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 aBOOLEAN
type. -
Enumerate a subset of columns by using a comma-separated list of regular expressions.
To use this type of conversion, you must set theconverters
configuration property with theselector
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 |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
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 |
---|---|---|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
ROWID types
The following table describes how the connector maps ROWID (row address) data types.
Oracle data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
This data type is not supported when using Oracle XStream. |
|
n/a |
This data type is not supported. |
XML types
Support for |
The following table describes how the connector maps XMLTYPE data types.
Oracle data type | Literal type (schema type) | Semantic type (schema name) and Notes |
---|---|---|
|
|
|
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 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.
Custom converters
By default, the Debezium Oracle connector provides several CustomConverter
implementations specific to Oracle data types.
These custom converters provide alternative mappings for specific data types based on the connector configuration.
To add a CustomConverter
to the connector, follow the instructions in the Custom Converters documentation.
The Debezium Oracle connector provides the following custom converters:
NUMBER(1)
to Boolean
Beginning with version 23, Oracle database provides a BOOLEAN
logical data type.
In earlier versions, the database simulates a BOOLEAN
type by using a NUMBER(1)
data type, constrained with a value of 0
for false, or a value of 1
for true.
By default, when Debezium emits change events for source columns that use the NUMBER(1)
data type, it converts the data to the INT8
literal type.
If the default mapping for NUMBER(1)
data types does not meet your needs, you can configure the connector to use the logical BOOL
type when it emits these columns by configuring the NumberOneToBooleanConverter
, as shown in the following example:
NumberOneToBooleanConverter
configurationconverters=number-to-boolean
number-to-boolean.type=io.debezium.connector.oracle.converters.NumberOneToBooleanConverter
number-to-boolean.selector=.*.MY_TABLE.DATA
In the preceding example, the selector
property is optional.
The selector
property specifies a regular expression that designates which tables or columns the converter applies to.
If you omit the selector
property, when Debezium emits an event, every column with the NUMBER(1)
data type is converted to a field that uses the logical BOOL
type.
NUMBER
To Zero Scale
Oracle supports creating NUMBER
based columns with negative scale, that is, NUMBER(-2)
.
Not all systems can process negative scale values, so these values can result in processing problems in your pipeline.
For example, because Apache Avro does not support these values, problems can occur if Debezium converts events to Avro format.
Similarly, downstream consumers that do not support these values can also encounter errors.
converters=number-zero-scale
number-zero-scale.type=io.debezium.connector.oracle.converters.NumberToZeroScaleConverter
number-zero-scale.decimal.mode=precise
In the preceding example, the decimal.mode
property specifies how the connector emits decimal values.
This property is optional.
If you omit the decimal.mode
property, the converter defaults to using the PRECISE
decimal handling mode.
RAW
to String
Although Oracle recommends against the use of certain data types, such as RAW
, legacy systems might continue to use such types.
By default, Debezium emits RAW
column types as logical BYTES
, a type that enables the storage of binary or text-based data.
In some cases, RAW
columns might store character data as a series of bytes.
To facilitate consumption by consumers, you can configure Debezium to use the RawToStringConverter
.
The RawToStringConverter
provides a way to easily target such RAW
columns and emit values as strings, rather than bytes.
The following example shows how to add the RawToStringConverter
to the connector configuration:
RawToStringConverter
configurationconverters=raw-to-string
raw-to-string.type=io.debezium.connector.oracle.converters.RawToStringConverter
raw-to-string.selector=.*.MY_TABLE.DATA
In the preceding example, the selector
property enables you to define a regular expression that specifies the tables or columns that the converter processes.
If you omit the selector
property, the converter maps all RAW
column types to logical STRING
field types.
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
-
vecsys
(Oracle 23+) -
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.
Tables excluded from capture
When the Debezium Oracle connector captures tables, it automatically excludes tables that match the following rules:
-
Compression Advisor tables matching the pattern
CMP[3|4]$[0-9]+
. -
Index-organized tables matching the pattern
SYS_IOT_OVER_%
. -
Spatial tables matching the patterns
MDRT_%
,MDRS_%
, orMDXT_%
. -
Nested tables
To enable the connector to capture a table with a name that matches any of the preceding rules, you must rename the table.
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;
Oracle AWS RDS does not allow you to execute the commands above nor does it allow you to log in as sysdba. AWS provides these alternative commands to configure LogMiner. Before executing these commands, ensure that your Oracle AWS RDS instance is enabled for backups.
To confirm that Oracle has backups enabled, execute the command below first. The LOG_MODE should say ARCHIVELOG. If it does not, you may need to reboot your Oracle AWS RDS instance.
SQL> SELECT LOG_MODE FROM V$DATABASE;
LOG_MODE
------------
ARCHIVELOG
Once LOG_MODE is set to ARCHIVELOG, execute the commands to complete LogMiner configuration. The first command set the database to archivelogs and the second adds supplemental logging.
exec rdsadmin.rdsadmin_util.set_configuration('archivelog retention hours',24);
exec rdsadmin.rdsadmin_util.alter_supplemental_logging('ADD');
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.
Archive log destinations
Oracle database administrators can configure up to 31 different destinations for archive logs.
Administrators can set parameters for each destination to designate it for a specific use, for example, log shipping for physical standbys, or external storage to allow for extended log retention.
Oracle reports details about archive log destinations in the V$ARCHIVE_DEST_STATUS
view.
The Debezium Oracle connector only uses destinations that have a status of VALID
and a type of LOCAL
.
If your Oracle environment includes multiple destinations that satisfy that criteria, consult with your Oracle administrator to determine which archive log destination Debezium should use.
-
To specify the archive log destination that you want Debezium to use, set the
archive.destination.name
property in the connector configuration.
For example, suppose that a database is configured with two archive destination paths,/path/one
and/path/two
, and that theV$ARCHIVE_DEST_STATUS
table associates these paths with destination names that are specified in the columnDEST_NAME
. If both destinations satisfy the criteria for Debezium — that is, theirstatus
isVALID
and theirtype
isLOCAL
— to configure the connector to use the archive logs that the database writes to/path/two
, set the value ofarchive.destination.name
to the value in theDEST_NAME
column that is associated with/path/two
in theV$ARCHIVE_DEST_STATUS
table. For example, if theDEST_NAME
isLOG_ARCHIVE_DEST_3
for/path/two
, you would configure Debezium as follows:
{
"archive.destination.name": "LOG_ARCHIVE_DEST_3"
}
Do not set the value of |
If your Oracle environment includes multiple destinations that satisfy that criteria, and you fail to specify the preferred destination, the Debezium Oracle connector selects the destination path at random. Because the retention policy that is configured for each destination might differ, this can lead to errors if the connector selects a path from which the requested log data was deleted. |
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 |
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)
GRANT SELECT ON V_$MYSTAT TO c##dbzuser CONTAINER=ALL; (24)
GRANT SELECT ON V_$STATNAME TO c##dbzuser CONTAINER=ALL; (25)
exit;
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 |
4 |
FLASHBACK ANY TABLE |
Enables the connector to perform Flashback queries, which is how the connector performs the initial snapshot of data.
Optionally, rather than granting |
5 |
SELECT ANY TABLE |
Enables the connector to read any table.
Optionally, rather than granting |
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 |
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 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 |
14 |
EXECUTE ON DBMS_LOGMNR_D |
Enables the connector to run methods in the |
15 to 25 |
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
A standby database provides a synchronized copy of the primary instance. In the event of a primary database failure, standby databases provide for continuous availability, and disaster recovery. Oracle makes use of both physical and logical standby databases.
A physical standby is an exact, block-for-block copy of the primary production database, and its system change number (SCN) values are identical to those of the primary. The Debezium Oracle connector cannot capture change events directly from a physical standby database, because physical standbys do not accept external connections. The connector can capture events from a physical standby only after the standby is converted to the primary database. The connector then connects to the former standby as if it were any primary database.
A logical standby contains the same logical data as the primary, but data might be stored in a different physical manner. SCN offsets in a logical standby differ from the offsets in the primary database. You can configure the Debezium Oracle connector to capture changes from a logical standby database.
Failover databases
When you set up a failover database, it is generally best practice to use a physical standby database rather than a logical standby database. A physical standby maintains a more consistent state with the primary database than does a logical standby. Physical standbys contain an exact replica of the primary data, and the system change number (SCN) values of the standby are identical to those of the primary. In a Debezium environment, after the database fails over to physical standby, the presence of consistent SCN values ensure that the connector can find the last processed SCN value.
A physical standby is locked in a read-only mode, with managed recovery running to maintain synchronization. When a database is in standby mode, it does not accept external JDBC connections from clients, and it cannot be accessed by external application.
After a failure event, to permit Debezium to connect to the former physical standby,a DBA must perform several actions to enable failover to the standby, and promote it the the primary database. The following list identifies some of the key actions:
-
Cancel managed recovery on the standby.
-
Complete the active recovery process.
-
Convert the standby to the primary role.
-
Open the new primary to client read and write operations.
After the former physical standby is available for normal use, you can configure the Debezium Oracle connector to connect to it. To enable the connector to capture from the new primary, edit the database hostname in the connector configuration, replacing the hostname of the original primary with the hostname of the new primary.
Configuring the Debezium Oracle connector to capture events from a logical standby
When the Debezium connector for Oracle connects to a primary database, it uses an internal flush table to manage the flush cycles of the Oracle Log Writer Buffer (LGWR) process. The flush process requires that the user account through which the connector accesses the database has permission to create and write to this flush table. However, a logical stand-by database typically permits read-only access, preventing the connector from writing to the database. You can modify the connector configuration to enable the connector to capture events from a logical standby, or the DBA can create a new writable tablespace in which the connector can store the flush table.
The ability to use the connector with a logical standby in an incubating state and can change without notice. There is an open Jira issue to investigate support for capturing changes from a physical standby. |
-
To enable Debezium to capture events from an Oracle read-only logical standby database, add the following property to the connector configuration, to disable creation and management of the flush table:
internal.log.mining.read.only=true
The preceding setting prevents the database from creating and updating the
LOG_MINING_FLUSH
table. You can use theinternal.log.mining.read.only
property with an Oracle Standalone database, or with an Oracle RAC installation.
Extended max string size
The database parameter max_string_size
controls how the Oracle database, and by extension, Debezium, interprets values for VARCHAR2
, NVARCHAR2
, and RAW
fields.
The default, STANDARD
, means the lengths for these data types align with the same limits with releases prior to Oracle 12c (4000 bytes for VARCHAR2
and NVARCHAR2
and 2000 bytes for RAW
).
When configured as EXTENDED
, these columns now allow up to 32767 bytes of data to be stored.
While a database administrator can change the |
For the Debezium Oracle connector, when the database parameter max_string_size
is EXTENDED
, the lob.enabled
connector configuration option should be set to true
to capture changes made to VARCHAR2
and NVARCHAR2
fields that have string lengths that exceed 4000 bytes or RAW
fields with more than 2000 bytes.
When set to EXTENDED
, Oracle performs an implicit conversion of the character data in-flight when the byte length of the string data exceeds the legacy maximums.
This implicit conversion means that Oracle internally treats the string data as though it is a CLOB
, so you get the benefit of treating the field as a regular string to the outside world, but with all the pitfalls and concerns about storage at the database tier level.
Since Oracle treats these strings as CLOB
internally, the redo logs also reflect several unique operation types that the Debezium Oracle connector needs to be aware that it should mine.
And because these operation types very closely resemble CLOB
operations, the redo log entries must be mined in the same way as any other LOB type, which is why lob.enabled
must be set to true
to capture changes from the redo logs regardless of the string data’s byte length.
When storing strings when |
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.
-
Download the Debezium Oracle connector plug-in archive.
-
Extract the files into your Kafka Connect environment.
-
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.
If you use the Debezium Oracle connector with Oracle XStream, obtain the JDBC driver as part of the Oracle Instant Client package. For more information, see Obtaining the Oracle JDBC driver and XStream API files. -
Download the XDB library for Oracle from Maven Central and extract the downloaded file to the directory that contains the Debezium Oracle connector JAR file.
-
Add the directory with the JAR files to Kafka Connect’s
plugin.path
. -
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.
{
"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)
"topic.prefix" : "server1", (8)
"tasks.max" : "1", (9)
"database.pdb.name" : "ORCLPDB1", (10)
"schema.history.internal.kafka.bootstrap.servers" : "kafka:9092", (11)
"schema.history.internal.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 | Topic prefix 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 schema history topic. |
12 | The name of the database schema 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 Transparent Network Substrate (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",
"topic.prefix" : "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",
"schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
"schema.history.internal.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.
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.
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",
"topic.prefix" : "server1",
"database.hostname" : "<oracle ip>",
"database.port" : "1521",
"database.user" : "c##dbzuser",
"database.password" : "dbz",
"database.dbname" : "ORCLCDB",
"database.pdb.name" : "ORCLPDB1",
"schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
"schema.history.internal.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",
"topic.prefix" : "server1",
"database.hostname" : "<oracle ip>",
"database.port" : "1521",
"database.user" : "c##dbzuser",
"database.password" : "dbz",
"database.dbname" : "ORCLCDB",
"schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
"schema.history.internal.kafka.topic": "schema-changes.inventory"
}
}
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 schema history connector configuration properties that control how Debezium processes events that it reads from the database schema history topic.
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 |
|||
No default |
Enumerates a comma-separated list of the symbolic names of the custom converter instances that the connector can use. For each converter that you configure for a connector, you must also add a
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. boolean.selector: .*MYTABLE.FLAG,.*.IS_ARCHIVED |
|||
|
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. In a container database environment, specify the name of the root container database (CDB), not the name of an included pluggable database (PDB). |
|||
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 |
Topic prefix that 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 topic prefix that is unique among all connectors in your Debezium environment.
The following characters are valid: alphanumeric characters, hyphens, dots, 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 except when
For more information, see the table of |
|||
false |
If the |
|||
false |
If the |
|||
false |
If the |
|||
false |
If the |
|||
false |
If the |
|||
No default |
If |
|||
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:
|
|||
No default |
When |
|||
|
Specifies how the connector queries data while performing a snapshot.
This setting enables you to manage snapshot content in a more flexible manner compared to using the |
|||
No default |
When |
|||
All tables specified in the connector’s |
An optional, comma-separated list of regular expressions that match the fully-qualified names ( In a multitenant container database (CDB) environment, the regular expression must include the pluggable database (PDB) name, using the format To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name. A snapshot can only include tables that are named in the connector’s This property takes effect only if the connector’s |
|||
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 From a "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 |
|||
No default |
An optional, comma-separated list of regular expressions that match names of schemas for which you want to capture changes.
In environments that use the LogMiner implementation, you must use POSIX regular expressions only.
Any schema name not included in To match the name of a schema, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the schema; it does not match substrings that might be present in a schema name. |
|||
|
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.
In environments that use the LogMiner implementation, you must use POSIX regular expressions only. To match the name of a schema, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the schema; it does not match substrings that might be present in a schema name. |
|||
No default |
An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be captured.
If you use the LogMiner implementation, use only POSIX regular expressions with this property.
When this property is set, the connector captures changes only from the specified tables.
Each table identifier uses the following format: To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name. |
|||
No default |
An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be excluded from monitoring.
If you use the LogMiner implementation, use only POSIX regular expressions with this property.
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: To match the name of a table, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the table; it does not match substrings that might be present in a table name. |
|||
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.
In environments that use the LogMiner implementation, you must use POSIX regular expressions only.
Fully-qualified names for columns use the following format: To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the column it does not match substrings that might be present in a column name. |
|||
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.
In environments that use the LogMiner implementation, you must use POSIX regular expressions only.
Fully-qualified column names use the following format: To match the name of a column, Debezium applies the regular expression that you specify as an anchored regular expression.
That is, the specified expression is matched against the entire name string of the column it does not match substrings that might be present in a column name. |
|||
|
Specifies whether to skip publishing messages when there is no change in included columns. This would essentially filter messages if there is no change in columns included as per |
|||
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 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. |
|||
bytes |
Specifies how binary ( |
|||
none |
Specifies how schema names should be adjusted for compatibility with the message converter used by the connector. Possible settings:
|
|||
none |
Specifies how field names should be adjusted for compatibility with the message converter used by the connector. Possible settings:
See Avro naming for more details. |
|||
|
Specifies how the connector should handle floating point values for
|
|||
|
Specifies how the connector should handle 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 each batch of events to process during each iteration of this connector. |
|||
|
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 |
|||
|
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. |
|||
|
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 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. |
|||
No default |
An optional, comma-separated list of regular expressions that match the fully-qualified names of character-based columns.
Set this property if you want the connector to mask the values for a set of columns, for example, if they contain sensitive data.
Set The fully-qualified name of a column observes the following format: You can specify multiple properties with different lengths in a single configuration. |
|||
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 for which you want the connector to emit extra parameters that represent column metadata. When this property is set, the connector adds the following fields to the schema of event records:
These parameters propagate a column’s original type name and length (for variable-width types), respectively. The fully-qualified name of a column observes one of the following formats: |
|||
No default |
An optional, comma-separated list of regular expressions that specify the fully-qualified names of data types that are defined for columns in a database. When this property is set, for columns with matching data types, the connector emits event records that include the following extra fields in their schema:
These parameters propagate a column’s original type name and length (for variable-width types), respectively. The fully-qualified name of a column observes one of the following formats: For the list of Oracle-specific data type names, see the Oracle data type mappings. |
|||
|
Specifies, in milliseconds, how frequently the connector sends messages to a heartbeat topic. |
|||
No default |
Specifies a query that the connector executes on the source database when the connector sends 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. |
|||
No default |
Specifies an interval in milliseconds that the connector waits after it starts before it takes a snapshot. |
|||
0 |
Specifies the time, in milliseconds, that the connector delays the start of the streaming process after it completes a snapshot.
Setting a delay interval helps to prevent the connector from restarting snapshots in the event that a failure occurs immediately after the snapshot completes, but before the streaming process begins.
Set a delay value that is higher than the value of the |
|||
|
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 the number of rows that will be fetched for each database round-trip of a given query.
Using a value of |
|||
|
Set the property to See Transaction Metadata for additional details. |
|||
|
Specifies the mining strategy that controls how Oracle LogMiner builds and uses a given data dictionary for resolving table and column ids to names. |
|||
|
Specifies the mining query mode that controls how the Oracle LogMiner query is built. |
|||
|
The buffer type controls how the connector manages buffering transaction data. |
|||
|
The maximum number of events a transaction is capable of having in the transaction buffer. Transactions with event counts that exceed this threshold not be emitted and will be abandoned. The default behavior is there is no transaction event threshold. |
|||
No default |
The XML configuration for the Infinispan global configuration. For more information, see Infinispan event buffering. |
|||
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. |
|||
|
Specifies whether the buffer state is deleted after the connector stops in a graceful, expected way. |
|||
|
The maximum number of milliseconds that a LogMiner session can be active before a new session is used. |
|||
|
Specifies whether the JDBC connection will be closed and re-opened on log switches or when mining session has reached maximum lifetime threshold. |
|||
|
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. This also servers as a measure for adjusting batch size - when the difference between current SCN and beginning/end SCN of the batch is bigger than this value, batch size is increased/decreased. |
|||
|
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 hours in the past from SYSDATE to mine archive logs.
When the default setting ( |
|||
|
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). |
|||
|
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 |
|||
|
Positive integer value that specifies the number of milliseconds to retain long running transactions between redo log switches.
When set to By default, 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. |
|||
No default |
Specifies the configured Oracle archive destination to use when mining archive logs with LogMiner. |
|||
No default |
List of database users to include from the LogMiner query. It can be useful to set this property if you want the capturing process to include changes from the specified users. |
|||
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. |
|||
|
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 |
|||
|
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 |
|||
|
Specifies the name of the flush table that coordinates flushing the Oracle LogWriter Buffer (LGWR) to the redo logs.
This name can be specified using the format |
|||
|
Specifies whether the redo log constructed SQL statement is included in |
|||
|
Controls whether or not large object (CLOB or BLOB) column values are emitted in change events. |
|||
|
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 compatibility with an Oracle RAC deployment. Specify the list of RAC nodes by using one of the following methods:
If you supply a raw JDBC URL for the database by using the |
|||
|
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:
By default, only truncate operations are skipped. |
|||
No default value |
Fully-qualified name of the data collection that is used to send signals to the connector.
When you use this property with an Oracle pluggable database (PDB), set its value to the name of the root database. |
|||
source |
List of the signaling channel names that are enabled for the connector. By default, the following channels are available:
|
|||
No default |
List of notification channel names that are enabled for the connector. By default, the following channels are available:
|
|||
|
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. |
|||
|
Specifies the watermarking mechanism that the connector uses during an incremental snapshot to deduplicate events that might be captured by an incremental snapshot and then recaptured after streaming resumes.
|
|||
|
The name of the TopicNamingStrategy class that should be used to determine the topic name for data change, schema change, transaction, heartbeat event etc., defaults to |
|||
|
Specify the delimiter for topic name, defaults to |
|||
|
The size used for holding the topic names in bounded concurrent hash map. This cache will help to determine the topic name corresponding to a given data collection. |
|||
|
Controls the name of the topic to which the connector sends heartbeat messages. The topic name has this pattern: |
|||
|
Controls the name of the topic to which the connector sends transaction metadata messages. The topic name has this pattern: |
|||
|
Specifies the number of threads that the connector uses when performing an initial snapshot. To enable parallel initial snapshots, set the property to a value greater than 1. In a parallel initial snapshot, the connector processes multiple tables concurrently. This feature is incubating. |
|||
|
Specifies the number of retry attempts to snapshot a table when a database error occurs.
This configuration property currently only retries failures related to |
|||
|
Defines tags that customize MBean object names by adding metadata that provides contextual information.
Specify a comma-separated list of key-value pairs.
Each key represents a tag for the MBean object name, and the corresponding value represents a value for the key, for example, The connector appends the specified tags to the base MBean object name. Tags can help you to organize and categorize metrics data. You can define tags to identify particular application instances, environments, regions, versions, and so forth. For more information, see Customized MBean names. |
|||
|
Specifies how the connector responds after an operation that results in a retriable error, such as a connection error.
|
|||
|
Time to wait for a query to execute, given in milliseconds. Defaults to 600 seconds (600,000 ms); zero means there is no limit. |
Debezium Oracle connector database schema history configuration properties
Debezium provides a set of schema.history.internal.*
properties that control how the connector interacts with the schema history topic.
The following table describes the schema.history.internal
properties for configuring the Debezium connector.
Property | Default | Description |
---|---|---|
No default |
The full name of the Kafka topic where the connector stores the database schema history. |
|
No default |
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. |
|
|
An integer value that specifies the maximum number of milliseconds the connector should wait while fetching cluster information using Kafka admin client. |
|
|
An integer value that specifies the maximum number of milliseconds the connector should wait while create kafka history topic using Kafka admin client. |
|
|
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 |
|
|
A Boolean value that specifies whether the connector records schema structures from all tables in a schema or database, or only from tables that are designated for capture.
|
|
|
A Boolean value that specifies whether the connector records schema structures from all logical databases in the database instance.
|
Pass-through Oracle connector configuration properties
The connector supports pass-through properties that enable Debezium to specify custom configuration options for fine-tuning the behavior of the Apache Kafka producer and consumer. For information about the full range of configuration properties for Kafka producers and consumers, see the Kafka documentation.
Pass-through properties for configuring how producer and consumer clients interact with schema history topics
Debezium relies on an Apache Kafka producer to write schema changes to database schema history topics.
Similarly, it relies on a Kafka consumer to read from database schema 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 schema.history.internal.producer.*
and schema.history.internal.consumer.*
prefixes.
The pass-through producer and consumer database schema history properties control a range of behaviors, such as how these clients secure connections with the Kafka broker, as shown in the following example:
schema.history.internal.producer.security.protocol=SSL
schema.history.internal.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
schema.history.internal.producer.ssl.keystore.password=test1234
schema.history.internal.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
schema.history.internal.producer.ssl.truststore.password=test1234
schema.history.internal.producer.ssl.key.password=test1234
schema.history.internal.consumer.security.protocol=SSL
schema.history.internal.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
schema.history.internal.consumer.ssl.keystore.password=test1234
schema.history.internal.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
schema.history.internal.consumer.ssl.truststore.password=test1234
schema.history.internal.consumer.ssl.key.password=test1234
Debezium strips the prefix from the property name before it passes the property to the Kafka client.
For more information about Kafka producer configuration properties and Kafka consumer configuration properties, see the Apache Kafka documentation .
Pass-through properties for configuring how the Oracle connector interacts with the Kafka signaling topic
Debezium provides a set of signal.*
properties that control how the connector interacts with the Kafka signals topic.
The following table describes the Kafka signal
properties.
Property | Default | Description | ||
---|---|---|---|---|
<topic.prefix>-signal |
The name of the Kafka topic that the connector monitors for ad hoc signals.
|
|||
kafka-signal |
The name of the group ID that is used by Kafka consumers. |
|||
No default |
A list of the host and port pairs that the connector uses to establish its initial connection to the Kafka cluster. Each pair references the Kafka cluster that is used by the Debezium Kafka Connect process. |
|||
|
An integer value that specifies the maximum number of milliseconds that the connector waits when polling signals. |
|||
|
Specifies whether the Kafka consumer writes an offset commit after it reads a message from the signaling topic. The value that you assign to this property determines whether the connector can process requests that the signaling topic receives while the connector is offline. Choose one of the following settings:
|
Pass-through properties for configuring the Kafka consumer client for the signaling channel
The Debezium connector provides for pass-through configuration of the signals Kafka consumer.
Pass-through signals properties begin with the prefix signals.consumer.*
.
For example, the connector passes properties such as signal.consumer.security.protocol=SSL
to the Kafka consumer.
Debezium strips the prefixes from the properties before it passes the properties to the Kafka signals consumer.
Pass-through properties for configuring the Oracle connector sink notification channel
The following table describes properties that you can use to configure the Debezium sink notification
channel.
Property | Default | Description |
---|---|---|
No default |
The name of the topic that receives notifications from Debezium.
This property is required when you configure the |
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 driver.*
.
For example, the connector passes properties such as driver.foobar=false
to the JDBC URL.
Debezium strips the prefixes from the properties before it passes the properties 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.
Customized MBean names
Debezium connectors expose metrics via the MBean name for the connector. These metrics, which are specific to each connector instance, provide data about the behavior of the connector’s snapshot, streaming, and schema history processes.
By default, when you deploy a correctly configured connector, Debezium generates a unique MBean name for each of the different connector metrics. To view the metrics for a connector process, you configure your observability stack to monitor its MBean. But these default MBean names depend on the connector configuration; configuration changes can result in changes to the MBean names. A change to the MBean name breaks the linkage between the connector instance and the MBean, disrupting monitoring activity. In this scenario, you must reconfigure the observability stack to use the new MBean name if you want to resume monitoring.
To prevent monitoring disruptions that result from MBean name changes, you can configure custom metrics tags.
You configure custom metrics by adding the custom.metric.tags
property to the connector configuration.
The property accepts key-value pairs in which each key represents a tag for the MBean object name, and the corresponding value represents the value of that tag.
For example: k1=v1,k2=v2
.
Debezium appends the specified tags to the MBean name of the connector.
After you configure the custom.metric.tags
property for a connector, you can configure the observability stack to retrieve metrics associated with the specified tags.
The observability stack then uses the specified tags, rather than the mutable MBean names to uniquely identify connectors.
Later, if Debezium redefines how it constructs MBean names, or if the topic.prefix
in the connector configuration changes, metrics collection is uninterrupted,
because the metrics scrape task uses the specified tag patterns to identify the connector.
A further benefit of using custom tags, is that you can use tags that reflect the architecture of your data pipeline, so that metrics are organized in a way that suits you operational needs.
For example, you might specify tags with values that declare the type of connector activity, the application context, or the data source, for example, db1-streaming-for-application-abc
.
If you specify multiple key-value pairs, all of the specified pairs are appended to the connector’s MBean name.
The following example illustrates how tags modify the default MBean name.
By default, the Oracle connector uses the following MBean name for streaming metrics:
debezium.oracle:type=connector-metrics,context=streaming,server=<topic.prefix>
If you set the value of custom.metric.tags
to database=salesdb-streaming,table=inventory
, Debezium generates the following custom MBean name:
debezium.oracle:type=connector-metrics,context=streaming,server=<topic.prefix>,database=salesdb-streaming,table=inventory
Snapshot Metrics
The MBean is debezium.oracle:type=connector-metrics,context=snapshot,server=<topic.prefix>
.
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 snapshot metrics that are available.
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 captured 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 paused. |
|
|
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. Includes also time when snapshot was paused. |
|
|
The total number of seconds that the snapshot was paused. If the snapshot was paused several times, the paused time adds up. |
|
|
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. This metric is available if |
|
|
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 |
---|---|---|
|
The identifier of the current snapshot chunk. |
|
|
The lower bound of the primary key set defining the current chunk. |
|
|
The upper bound of the primary key set defining the current chunk. |
|
|
The lower bound of the primary key set of the currently snapshotted table. |
|
|
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=<topic.prefix>
.
The following table lists the streaming metrics that are available.
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 data change events reported by the source database since the last connector start, or since a metrics reset. Represents the data change workload for Debezium to process. |
|
|
The total number of create events processed by the connector since its last start or metrics reset. |
|
|
The total number of update events processed by the connector since its last start or metrics reset. |
|
|
The total number of delete events processed by the connector since its last start or metrics 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 captured 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 incorporate 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. This metric is available if |
|
|
The current volume, in bytes, of records in the queue. |
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 oldest system change number’s age in milliseconds.
If the buffer is empty, the value will be |
|
|
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.
For more information, see |
|
|
The number of current active transactions in the transaction buffer. |
|
|
The number of committed transactions in the transaction buffer. |
|
|
The number of transactions that were discarded because their size exceeded |
|
|
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 the most recent abandoned transaction identifiers removed from the transaction buffer due to their age.
See |
|
|
Current number of entries in the abandoned transactions list. |
|
|
An array of the most recent 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 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 |
|
|
The number of DDL records that have been detected but could not be parsed by the DDL parser.
This should always be |
|
|
The current mining session’s user global area (UGA) memory consumption in bytes. |
|
|
The maximum mining session’s user global area (UGA) memory consumption in bytes across all mining sessions. |
|
|
The current mining session’s process global area (PGA) memory consumption in bytes. |
|
|
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=<topic.prefix>
.
The following table lists the schema history metrics that are available.
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
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.
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 schema.history.internal.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 |
|
OpenLogReplicator support
The OpenLogReplicator ingestion adapter is currently in incubating state, i.e. exact semantics, configuration options etc. may change in future revisions, based on the feedback we receive. Please let us know if you encounter any problems. |
The Debezium Oracle connector by default ingests changes using native Oracle LogMiner. However, the connector can be toggled to use OpenLogReplicator, an open-source and free third-party application that reads Oracle changes directly from the redo and archive logs with low impact on the database. To configure the connector to use OpenLogReplicator, you must apply specific database and connector configurations that differ from those that you use with LogMiner.
-
Download and compile OpenLogReplicator for your database environment.
-
OpenLogReplicator must be installed with direct access to the archive and redo log files. This does not necessarily require installation on the physical database server if the archive and redo logs can be accessed via some shared filesystem.
How OpenLogReplicator works
OpenLogReplicator takes on the role of Oracle LogMiner and Oracle XStream when the Debezium Oracle connector streams changes. It is responsible for the capturing of changes in the redo and archive logs as they occur and batching those changes into logical transactions. Debezium Oracle connector is a consumer of OpenLogReplicator by connecting to the network endpoint provided by OpenLogReplicator and ingesting the transactions as they’re batched.
After the OpenLogReplicator adapter ingests changes, the Debezium Oracle connector transforms the events into data change events just like any other adapter.
From a network topology perspective, the Debezium Oracle connector relies on network connections to both the Oracle database and to the OpenLogReplicator. Similarly, OpenLogReplicator requires a network connection to the Oracle database, as well as direct access to the raw redo and archive logs.
Preparing the database
The Oracle database must be configured to generate specific files, called archive logs.
An archive log is a saved copy of a full online redo log.
The database must be placed in ARCHIVELOG
mode before it can archive redo log files.
To place the database in ARCHIVELOG
mode, you must set specific configuration properties to specify the destination for saving the archive log files.
The following example shows a command for placing a database into archive log mode:
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;
shutdown immediate
startup mount
alter database archivelog;
alter database open;
-- Should show "Database log mode: Archive Mode"
archive log list
exit;
For Debezium to generate change events that show the before
and after
states of a table row, supplemental logging must be active on the database.
Supplemental logging adds column data to the redo logs to identify the rows that are affected when a table is modified.
You can use different methods to configure supplemental logging. To support Debezium, you must enable at least minimal database-level supplemental logging. Minimal supplemental logging writes the least amount of information needed to be able to create change events. The following example shows a command that you might use to enable minimal supplemental logging.
ALTER DATABASE ADD SUPPLEMENTAL LOG DATA
Debezium for Oracle can work with a higher degree of supplemental logging at the database level, if that is already configured for other purposes. But if you do not require higher fidelity logging to support other applications, you can reduce database-level logging to the minimum level. |
For each captured table, you must explicitly configure a higher fidelity supplemental logging, called (ALL) COLUMNS
.
The (ALL) COLUMNS
logging level guarantees that Oracle captures the state of a column regardless of whether the column changed when a redo entry is written to the redo log.
Enabling the higher logging level enables Debezium for Oracle to generate change events that provide the accurate before and after states for a row.
The following example shows a command for enabling supplemental logging for each captured table.
ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS
Whenever new tables are added to the Debezium Oracle’s connector configuration, you must configure supplemental logging for each table. If a table that is configured for capture is not correctly configured for supplemental logging, after the connector begins streaming, it returns a warning message. |
Creating connector users
The Debezium Oracle connector requires a user account to be set up with specific permissions so that the connector can capture change events. This user account will be used by both Debezium and OpenLogReplicator. The following example shows a possible user account configuration for deploying Debezium with OpenLogReplicator in a multi-tenant Oracle environment.
# Create Log Miner Tablespace and User
sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba <<- EOF
CREATE TABLESPACE LOGMINER_TBS DATAFILE '/opt/oracle/oradata/ORCLCDB/logminer_tbs.dbf' SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
exit;
EOF
sqlplus sys/top_secret@//localhost:1521/ORCLPDB1 as sysdba <<- EOF
CREATE TABLESPACE LOGMINER_TBS DATAFILE '/opt/oracle/oradata/ORCLCDB/ORCLPDB1/logminer_tbs.dbf' SIZE 25M REUSE AUTOEXTEND ON MAXSIZE UNLIMITED;
exit;
EOF
sqlplus sys/top_secret@//localhost:1521/ORCLCDB as sysdba <<- EOF
CREATE USER c##dbzuser IDENTIFIED BY dbz DEFAULT TABLESPACE LOGMINER_TBS QUOTA UNLIMITED ON LOGMINER_TBS CONTAINER=ALL;
-- Debezium specific permissions
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 SELECT ANY DICTIONARY TO c##dbzuser CONTAINER=ALL;
GRANT LOCK ANY TABLE TO c##dbzuser CONTAINER=ALL;
-- These can be reduced from ANY TABLE to your captured tables depending on your security model
GRANT SELECT ANY TABLE TO c##dbzuser CONTAINER=ALL;
GRANT FLASHBACK ANY TABLE TO c##dbzuser CONTAINER=ALL;
-- OpenLogReplicator specific permissions
ALTER SESSION SET CONTAINER = ORCLPDB1;
GRANT SELECT, FLASHBACK ON SYS.CCOL$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.CDEF$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.COL$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.DEFERRED_STG$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.ECOL$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.LOB$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.LOBCOMPPART$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.LOBFRAG$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.OBJ$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.TAB$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.TABCOMPART$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.TABPART$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.TABSUBPART$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.TS$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON SYS.USER$ TO c##dbzuser;
GRANT SELECT, FLASHBACK ON XDB.XDB$TTSET TO c##dbzuser;
exit;
EOF
Configuring the OpenLogReplicator adapter
By default, Debezium uses Oracle LogMiner to ingest change events from Oracle. You can modify the default settings to configure the connector to use the OpenLogReplicator adapter in place of LogMiner.
In the example that follows, the following properties are added to the connector configuration to enable the connector to use the OpenLogReplicator adapter:
-
database.connection.adapter
-
openlogreplicator.source
-
openlogreplicator.host
-
openlogreplicator.port
{
"name": "inventory-connector",
"config": {
"connector.class" : "io.debezium.connector.oracle.OracleConnector",
"tasks.max" : "1",
"topic.prefix" : "server1",
"database.hostname" : "<oracle ip>",
"database.port" : "1521",
"database.user" : "c##dbzuser",
"database.password" : "dbz",
"database.dbname" : "ORCLCDB",
"database.pdb.name" : "ORCLPDB1",
"schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
"schema.history.internal.kafka.topic": "schema-changes.inventory",
"database.connection.adapter": "olr",
"openlogreplicator.source": "ORACLE",
"openlogreplicator.host": "<ip address or hostname of OpenLogReplicator>",
"openlogreplicator.port": "<port OpenLogReplicator is listening on>"
}
}
Building OpenLogReplicator
OpenLogReplicator does not distribute binaries, so you must build the third-party tool from code. OpenLogReplicator is written in C++; however, the author provides a container image to build the source for a variety of operating systems, including RHEL, Fedora, CentOS, Debian, and others. For information about how to build the tool using a container, see the This OpenLogReplicator GitHub repository.
Obtaining the Oracle JDBC driver for OpenLogReplicator
When you use the Debezium Oracle connector with the OpenLogReplicator, you must obtain the Oracle JDBC driver to connect to Oracle databases. For more information, see Obtaining the Oracle JDBC driver.
OpenLogReplicator configuration
OpenLogReplicator is a third-party standalone process that is responsible for reading the Oracle redo and archive logs. Unlike other Debezium Oracle adapters, this means that there are two configurable components that must be configured independently of one another.
You configure OpenLogReplicator by using a JSON file called scripts/OpenLogReplicator.json
.
For more information about the required format of this file, see the OpenLogReplicator documentation.
{
"version": "1.6.0",
"source": [{
"alias": "SOURCE",
"name": "ORACLE", (1)
"reader": {
"type": "online",
"path-mapping": ["/opt/olr/recovery_area", "/opt/olr/recovery_area/ORCLCDB/archivelog"], (2)
"user": "c##dbzuser", (3)
"password": "dbz", (4)
"server": "//<ip>:<port>/ORCLPDB1" (5)
},
"format": { (6)
"type": "json",
"column": 2,
"db": 3,
"interval-dts": 9,
"interval-ytm": 4,
"message": 2,
"rid": 1,
"schema": 7,
"scn-all": 1,
"timestamp-all": 1
}
}],
"target": [{
"alias": "DEBEZIUM",
"source": "SOURCE",
"writer": {
"type": "network", (7)
"uri": "<host>:<port>" (8)
}
}]
}
1 | This should match the openlogreplicator.source connector configuration. |
2 | List of file path pairs [before1,after1,befor2,after2,…] where if a log file path matches one of the beforeX prefixes, the prefix is replaced with the afterX path. This is useful when OpenLogReplicator is running on a different host than the source database and the path to the redo and archive logs different between the database and the OpenLogReplicator process. |
3 | This should match the database.user connector property. |
4 | This should match the database.password connector property. |
5 | This should point to the database host, port, and Oracle SID. |
6 | This specifies the payload format ingested by Debezium Oracle connector. Use these values as specified, as these are the only format options that we require beyond the defaults. |
7 | This must specify network , as Debezium Oracle connector communicates with OpenLogReplicator via a network connection. |
8 | This specifies the bind host and port that OpenLogReplicator listens for connections on. This should be accessible by Debezium Oracle connector. |
OpenLogReplicator connector properties
The following configuration properties are required when using OpenLogReplicator.
Property |
Default |
Description |
No default |
The logical name of the configured |
|
No default |
The host name or IP address of the OpenLogReplicator network service. |
|
No default |
The port number that is used by the OpenLogReplicator network service. |
OpenLogReplicator ROWID support
OpenLogReplicator does not provide the ROWID
pseudocolumn details in its change event payloads.
Consequently, in environments that use the OpenLogReplicator adapter, the Debezium connector for Oracle consistently displays empty row_id
values in the source information block.
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;
When setting up an XStream Outbound Server to capture changes from a pluggable database,
the |
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",
"topic.prefix" : "server1",
"database.hostname" : "<oracle ip>",
"database.port" : "1521",
"database.user" : "c##dbzuser",
"database.password" : "dbz",
"database.dbname" : "ORCLCDB",
"database.pdb.name" : "ORCLPDB1",
"schema.history.internal.kafka.bootstrap.servers" : "kafka:9092",
"schema.history.internal.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 (ojdbc11.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.
-
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 ... ├── ojdbc11.jar ├── ucp.jar ├── uidrvci └── xstreams.jar
-
Copy the
ojdbc11.jar
andxstreams.jar
files, and add them to the<kafka_home>/libs
directory, for example,kafka/libs
. -
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.
Frequently Asked Questions
- Is Oracle 11g supported?
-
Oracle 11g is not supported; however, we do aim to be backward compatible with Oracle 11g on a best-effort basis. We rely on the community to communicate compatibility concerns with Oracle 11g as well as provide bug fixes when a regression is identified.
- Isn’t Oracle LogMiner deprecated?
-
No, Oracle only deprecated the continuous mining option with Oracle LogMiner in Oracle 12c and removed that option starting with Oracle 19c. The Debezium Oracle connector does not rely on this option to function, and therefore can safely be used with newer versions of Oracle without any impact.
- How do I change the position in the offsets?
-
The Debezium Oracle connector maintains two critical values in the offsets, a field named
scn
and another namedcommit_scn
. Thescn
field is a string that represents the low-watermark starting position the connector used when capturing changes.-
Find out the name of the topic that contains the connector offsets. This is configured based on the value set as the
offset.storage.topic
configuration property. -
Find out the last offset for the connector, the key under which it is stored and identify the partition used to store the offset. This can be done using the
kafkacat
utility script provided by the Kafka broker installation. An example might look like this:kafkacat -b localhost -C -t my_connect_offsets -f 'Partition(%p) %k %s\n' Partition(11) ["inventory-connector",{"server":"server1"}] {"scn":"324567897", "commit_scn":"324567897: 0x2832343233323:1"}
The key for
inventory-connector
is["inventory-connector",{"server":"server1"}]
, the partition is11
and the last offset is the contents that follows the key. -
To move back to a previous offset the connector should be stopped and the following command has to be issued:
echo '["inventory-connector",{"server":"server1"}]|{"scn":"3245675000","commit_scn":"324567500"}' | \ kafkacat -P -b localhost -t my_connect_offsets -K \| -p 11
This writes to partition
11
of themy_connect_offsets
topic the given key and offset value. In this example, we are reversing the connector back to SCN3245675000
rather than324567897
.
-
- What happens if the connector cannot find logs with a given offset SCN?
-
The Debezium connector maintains a low and high -watermark SCN value in the connector offsets. The low-watermark SCN represents the starting position and must exist in the available online redo or archive logs in order for the connector to start successfully. When the connector reports it cannot find this offset SCN, this indicates that the logs that are still available do not contain the SCN and therefore the connector cannot mine changes from where it left off.
When this happens, there are two options. The first is to remove the history topic and offsets for the connector and restart the connector, taking a new snapshot as suggested. This will guarantee that no data loss will occur for any topic consumers. The second is to manually manipulate the offsets, advancing the SCN to a position that is available in the redo or archive logs. This will cause changes that occurred between the old SCN value and the newly provided SCN value to be lost and not written to the topics. This is not recommended.
- What’s the difference between the various mining strategies?
-
The Debezium Oracle connector provides three options for
log.mining.strategy
.The default is
redo_in_catalog
, and this instructs the connector to write the Oracle data dictionary to the redo logs everytime a log switch is detected. This data dictionary is necessary for Oracle LogMiner to track schema changes effectively when parsing the redo and archive logs. This option will generate more than usual numbers of archive logs but allows tables being captured to be manipulated in real-time without any impact on capturing data changes. This option generally requires more Oracle database memory and will cause the Oracle LogMiner session and process to take slightly longer to start after each log switch.The second option,
online_catalog
, does not write the data dictionary to the redo logs. Instead, Oracle LogMiner will always use the online data dictionary that contains the current state of the table’s structure. This also means that if a table’s structure changes and no longer matches the online data dictionary, Oracle LogMiner will be unable to resolve table or column names if the table’s structure is changed. This mining strategy option should not be used if the tables being captured are subject to frequent schema changes. It’s important that all data changes be lock-stepped with the schema change such that all changes have been captured from the logs for the table, stop the connector, apply the schema change, and restart the connector and resume data changes on the table. This option requires less Oracle database memory and Oracle LogMiner sessions generally start substantially faster since the data dictionary does not need to be loaded or primed by the LogMiner process.The final option,
hybrid
, combines the strengths of the above two strategies with none of their weaknesses. This strategy harnesses the performance of theonline_catalog
with the resilience in schema tracking of theredo_in_catalog
while also avoiding the overhead and performance costs with the higher than normal archive log generation. This mode utilizes a fallback mode where if LogMiner fails to reconstruct the SQL for a database change, the Debezium connector will rely on the in-memory schema model maintained by the connector to reconstruct the SQL in-flight. The intent is that this mode will eventually transition to the default, and likely only mode of operation in the future. - Are there any limitations with the Hybrid mining strategy with LogMiner?
-
Yes, the Hybrid mode for
log.mining.strategy
is still a work-in-progress strategy, and therefore does not yet support all data types. At this time, this mode cannot reconstruct SQL statements that include operations againstCLOB
,NCLOB
,BLOB
,XML
, norJSON
data types. So in short, if you enablelob.enabled
with a value oftrue
, you will be unable to use the Hybrid strategy and the connector will fail to start as this combination is unsupported. - Why does the connector appear to stop capturing changes 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:
-
From a terminal, run the following command:
sysctl -w net.ipv4.tcp_keepalive_time=60
-
Edit
/etc/sysctl.conf
and set the value of the following variable as shown:net.ipv4.tcp_keepalive_time=60
-
Reconfigure the Debezium for Oracle connector to use the
database.url
property rather thandatabase.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.
-
- What’s the cause for ORA-01555 and how to handle it?
-
The Debezium Oracle connector uses flashback queries when the initial snapshot phase executes. A flashback query is a special type of query that relies on the flashback area, maintained by the database’s
UNDO_RETENTION
database parameter, to return the results of a query based on what the contents of the table had at a given time, or in our case at a given SCN. By default, Oracle generally only maintains an undo or flashback area for approximately 15 minutes unless this has been increased or decreased by your database administrator. For configurations that capture large tables, it may take longer than 15 minutes or your configuredUNDO_RETENTION
to perform the initial snapshot and this will eventually lead to this exception:ORA-01555: snapshot too old: rollback segment number 12345 with name "_SYSSMU11_1234567890$" too small
The first way to deal with this exception is to work with your database administrator and see whether they can increase the
UNDO_RETENTION
database parameter temporarily. This does not require a restart of the Oracle database, so this can be done online without impacting database availability. However, changing this may still lead to the above exception or a "snapshot too old" exception if the tablespace has inadequate space to store the necessary undo data.The second way to deal with this exception is to not rely on the initial snapshot at all, setting the
snapshot.mode
toschema_only
and then instead relying on incremental snapshots. An incremental snapshot does not rely on a flashback query and therefore isn’t subject to ORA-01555 exceptions. - What’s the cause for ORA-04036 and how to handle it?
-
The Debezium Oracle connector may report an ORA-04036 exception when the database changes occur infrequently. An Oracle LogMiner session is started and re-used until a log switch is detected. The session is re-used as it provides the optimal performance utilization with Oracle LogMiner, but should a long-running mining session occur, this can lead to excessive PGA memory usage, eventually causing an exception like this:
ORA-04036: PGA memory used by the instance exceeds PGA_AGGREGATE_LIMIT
This exception can be avoided by specifying how frequent Oracle switches redo logs or how long the Debezium Oracle connector is allowed to re-use the mining session. The Debezium Oracle connector provides a configuration option,
log.mining.session.max.ms
, which controls how long the current Oracle LogMiner session can be re-used for before being closed and a new session started. This allows the database resources to be kept in-check without exceeding the PGA memory allowed by the database. - What’s the cause for ORA-01882 and how to handle it?
-
The Debezium Oracle connector may report the following exception when connecting to an Oracle database:
ORA-01882: timezone region not found
This happens when the timezone information cannot be correctly resolved by the JDBC driver. In order to solve this driver related problem, the driver needs to be told to not resolve the timezone details using regions. This can be done by specifying a driver pass through property using
driver.oracle.jdbc.timezoneAsRegion=false
. - What’s the cause for ORA-25191 and how to handle it?
-
The Debezium Oracle connector automatically ignores index-organized tables (IOT) as they are not supported by Oracle LogMiner. However, if an ORA-25191 exception is thrown, this could be due to a unique corner case for such a mapping and the additional rules may be necessary to exclude these automatically. An example of an ORA-25191 exception might look like this:
ORA-25191: cannot reference overflow table of an index-organized table
If an ORA-25191 exception is thrown, please raise a Jira issue with the details about the table and it’s mappings, related to other parent tables, etc. As a workaround, the include/exclude configuration options can be adjusted to prevent the connector from accessing such tables.
- How to solve SAX feature external-general-entities not supported
-
Debezium 2.4 introduced support for Oracle’s
XMLTYPE
column type and to support this feature, the Oraclexdb
andxmlparserv2
dependencies are required.
Oracle’sxmlparserv2
dependency implements a SAX-based parser and if the runtime finds an uses this implementation rather than the other on the classpath, this error will occur. In order to influence specifically which SAX implementation is used generally, the JVM will need to be started with a specific argument.
When the following JVM argument is provided, the Oracle connector will start successfully without this error.
-Djavax.xml.parsers.SAXParserFactory=com.sun.org.apache.xerces.internal.jaxp.SAXParserFactoryImpl