{prodname}'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 link:https://debezium.io/releases/[{prodname} release overview].
For information about the Oracle Database versions that are compatible with this connector, see the link:{LinkDebeziumSupportedConfigurations}[{NameDebeziumSupportedConfigurations}].
To optimally configure and run a {prodname} Oracle connector, it is helpful to understand how the connector performs snapshots, streams change events, determines Kafka topic names, and uses metadata.
Typically, the redo logs on an Oracle server are configured to not retain the complete history of the database.
As a result, the {prodname} Oracle connector cannot retrieve the entire history of the database from the logs.
To enable the connector to establish a baseline for the current state of the database, the first time that the connector starts, it performs an initial _consistent snapshot_ of the database.
You can customize the way that the connector creates snapshots by setting the value of the xref:oracle-property-snapshot-mode[`snapshot.mode`] connector configuration property.
.Default connector workflow for creating an initial snapshot
When the snapshot mode is set to the default, the connector completes the following tasks to create a snapshot:
1. Determines the tables to be captured
2. Obtains a `ROW SHARE MODE` lock on each of the monitored tables to prevent structural changes from occurring during creation of the snapshot. {prodname} holds the locks for only a short time.
3. Reads the current system change number (SCN) position from the server's redo log.
4. Captures the structure of all relevant tables.
5. Releases the locks obtained in Step 2.
6. Scans all of the relevant database tables and schemas as valid at the SCN position that was read in Step 3 (`SELECT * FROM ... AS OF SCN 123`), generates a `READ` event for each row, and then writes the event records to the table-specific Kafka topic.
7. Records the successful completion of the snapshot in the connector offsets.
After the snapshot process begins, if the process is interrupted due to connector failure, rebalancing, or other reasons, the process restarts after the connector restarts.
After the connector completes the initial snapshot, it continues streaming from the position that it read in Step 3 so that it does not miss any updates.
If the connector stops again for any reason, after it restarts, it resumes streaming changes from where it previously left off.
|The connector performs a database snapshot as described in the xref:default-workflow-for-performing-an-initial-snapshot[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.
|The connector captures the structure of all relevant tables, performing all of the steps described in the xref:default-workflow-for-performing-an-initial-snapshot[default snapshot workflow], except that it does not create `READ` events to represent the data set at the point of the connector's start-up (Step 6).
|Set this option to restore a database history topic that is lost or corrupted.
After a restart, the connector runs a snapshot that rebuilds the topic from the source tables.
You can also set the property to periodically prune the database history topic that experience unexpected growth.
Note this mode is only safe to be used when it is guaranteed that no schema changes happened since the point in time the connector was shut down before and the point in time the snapshot is taken.
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:
The following list provides definitions for the components of the default name:
_serverName_:: The logical name of the server as specified by the xref:oracle-property-database-server-name[`database.server.name`] connector configuration property.
_schemaName_:: The name of the schema in which the operation occurred.
_tableName_:: The name of the table in which the operation occurred.
For example, if `fulfillment` is the server name, `inventory` is the schema name, and the database contains tables with the names `orders`, `customers`, and `products`,
the {prodname} Oracle connector emits events to the following Kafka topics, one for each table in the database:
The connector applies similar naming conventions to label its internal database history topics, xref:oracle-schema-change-topic[schema change topics], and xref:oracle-transaction-metadata[transaction metadata topics].
You can configure a {prodname} Oracle connector to produce schema change events that describe schema changes that are applied to captured tables in the database.
The connector writes schema change events to a Kafka topic named `_<serverName>_`, where `_serverName_` is the logical server name that is specified in the xref:oracle-property-database-server-name[`database.server.name`] configuration property.
{prodname} emits a new message to this topic whenever it streams data from a new table.
Messages that the connector sends to the schema change topic contain a payload, and, optionally, also contain the schema of the change event message.
The payload of a schema change event message includes the following elements:
`ddl`:: Provides the SQL `CREATE`, `ALTER`, or `DROP` statement that results in the schema change.
`databaseName`:: The name of the database to which the statements are applied.
`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.
[IMPORTANT]
====
When the connector is configured to capture a table, it stores the history of the table's schema changes not only in the schema change topic, but also in an internal database history topic.
The internal database history topic is for connector use only and it is not intended for direct use by consuming applications.
Ensure that applications that require notifications about schema changes consume that information only from the schema change topic.
====
[IMPORTANT]
====
Never partition the database history topic.
For the database history topic to function correctly, it must maintain a consistent, global order of the event records that the connector emits to it.
To ensure that the topic is not split among partitions, set the partition count for the topic by using one of the following methods:
* If you create the database history topic manually, specify a partition count of `1`.
* If you use the Apache Kafka broker to create the database history topic automatically, the topic is created, set the value of the link:{link-kafka-docs}/#brokerconfigs_num.partitions[Kafka `num.partitions`] configuration option to `1`.
`data_collections` (for `END` events):: An array of pairs of `data_collection` and `event_count` elements that indicates number of events that the connector emits for changes that originate from a data collection.
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 xref:oracle-property-log-mining-buffer-type[`log.mining.buffer.type`].
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.
The {prodname} 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 xref:oracle-property-log-mining-buffer-type[`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 xref:oracle-connector-properties[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:
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 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 {prodname} 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.
The {prodname} 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:
When the {prodname} 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 because of large or bulk transactions in the database.
Under certain circumstances, the Oracle database advances the system change number by an unusually high amount, rather than increasing it 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.
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.
Additionally, the connector will ignore the mining maximum batch size for this iteration only when this occurs.
The {prodname} Oracle connector ensures that all Kafka Connect _schema names_ are http://avro.apache.org/docs/current/spec.html#names[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.
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.
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`.
Although the `column.exclude.list` configuration property allows you to remove columns from the event values, all columns in a primary or unique key are always included in the event's key.
If the table does not have a primary or unique key, then the change event's key is null. This makes sense since the rows in a table without a primary or unique key constraint cannot be uniquely identified.
`op`:: A mandatory field that contains a string value describing the type of operation. Values for the Oracle connector are `c` for create (or insert), `u` for update, `d` for delete, and `r` for read (in the case of a snapshot).
`before`:: An optional field that, if present, contains the state of the row _before_ the event occurred. The structure is described by the `server1.INVENTORY.CUSTOMERS.Value` Kafka Connect schema, which the `server1` connector uses for all rows in the `inventory.customers` table.
// Whether or not this field and its elements are available is highly dependent on the https://docs.oracle.com/database/121/SUTIL/GUID-D2DDD67C-E1CC-45A6-A2A7-198E4C142FA3.htm#SUTIL1583[Supplemental Logging] configuration applying to the table.
`after`:: An optional field that if present contains the state of the row _after_ the event occurred. The structure is described by the same `server1.INVENTORY.CUSTOMERS.Value` Kafka Connect schema used in `before`.
`source`:: A mandatory field that contains a structure describing the source metadata for the event, which in the case of Oracle contains these fields: the {prodname} version, the connector name, whether the event is part of an ongoing snapshot or not, the transaction id (not while snapshotting), the SCN of the change, and a timestamp representing the point in time when the record was changed in the source database (during snapshotting, this is the point in time of snapshotting).
`ts_ms`:: An optional field that, if present, contains the time (using the system clock in the JVM running the Kafka Connect task) at which the connector processed the event.
And of course, the _schema_ portion of the event message's value contains a schema that describes this envelope structure and the nested fields within it.
The names of the schemas for the `before` and `after` fields are of the form `_<logicalName>_._<schemaName>_._<tableName>_.Value`, and thus are entirely independent from the schemas for all other tables.
This means that when using the xref:{link-avro-serialization}#avro-serialization[Avro Converter], the resulting Avro schems for _each table_ in each _logical source_ have their own evolution and history.
The `payload` portion of this event's _value_, provides information about the event.
It describes that a row was created (`op=c`), and shows that the `after` field value contains the values that were inserted into the `ID`, `FIRST_NAME`, `LAST_NAME`, and `EMAIL` columns of the row.
The increased size results from the JSON representation including both the schema and payload portions of a message.
You can use the xref:{link-avro-serialization}#avro-serialization[Avro Converter] to decrease the size of messages that the connector writes to Kafka topics.
The value of an _update_ change event on this table has the same _schema_ as the _create_ event. The payload uses the same structure, but it holds different values.
* The `op` field value is now `u`, signifying that this row changed because of an update
* The `before` field now has the state of the row with the values before the database commit
* The `after` field now has the updated state of the row, and here was can see that the `EMAIL` value is now `anne@example.com`.
* The `source` field structure has the same fields as before, but the values are different since this event is from a different position in the redo log.
Now, let's look at the value of a _delete_ event for the same table. As is the case with _create_ and _update_ events, for a `_delete_` event, the `schema` portion of the value is exactly the same:
When a row is deleted, the _delete_ event value listed above still works with log compaction, since Kafka can still remove all earlier messages with that same key.
The message value must be set to `null` to instruct Kafka to remove _all messages_ that share the same key.
To make this possible, by default, {prodname}'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 xref:oracle-property-tombstones-on-delete[`tombstones.on.delete`].
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:
[source,json,indent=0,subs="+attributes"]
----
{
"schema": { ... },
"payload": {
"before": null,
"after": null,
"source": { // <1>
"version": "{debezium-version}",
"connector": "oracle",
"name": "oracle_server",
"ts_ms": 1638974535000,
"snapshot": "false",
"db": "ORCLPDB1",
"sequence": null,
"schema": "DEBEZIUM",
"table": "TEST_TABLE",
"txId": "02000a0037030000",
"scn": "13234397",
"commit_scn": "13271102",
"lcr_position": null
},
"op": "t", // <2>
"ts_ms": 1638974558961, // <3>
"transaction": null
}
}
----
.Descriptions of _truncate_ event value fields
[cols="1,2,7",options="header"]
|===
|Item |Field name |Description
|1
|`source`
a|Mandatory field that describes the source metadata for the event. In a _truncate_ event value, the `source` field structure is the same as for _create_, _update_, and _delete_ events for the same table, provides this metadata:
* {prodname} version
* Connector type and name
* Database and table that contains the new row
* Schema name
* If the event was part of a snapshot (always `false` for _truncate_ events)
* ID of the transaction in which the operation was performed
* SCN of the operation
* Timestamp for when the change was made in the database
|2
|`op`
a|Mandatory string that describes the type of operation. The `op` field value is `t`, signifying that this table was truncated.
|3
|`ts_ms`
a|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 {prodname}.
|===
Note that since _truncate_ events represent a change made to an entire table and don't have a message key,
unless you're working with topics with a single partition,
there are no ordering guarantees for the change events pertaining to a table (_create_, _update_, etc.) and _truncate_ events for that table.
For instance a consumer may receive an _update_ event only after a _truncate_ event for that table,
when those events are read from different partitions.
If _truncate_ events are not desired, they can be filtered out with the xref:oracle-property-skipped-operations[`skipped.operations`] option.
To represent changes that occur in a table rows, the {prodname} Oracle connector emits change events that are structured like the table in which the rows exists.
The event contains a field for each column value.
Column values are represented according to the Oracle data type of the column.
The following sections describe how the connector maps oracle data types to a _literal type_ and a _semantic type_ in event fields.
_literal type_:: Describes how the value is literally represented using Kafka Connect schema types: `INT8`, `INT16`, `INT32`, `INT64`, `FLOAT32`, `FLOAT64`, `BOOLEAN`, `STRING`, `BYTES`, `ARRAY`, `MAP`, and `STRUCT`.
Support for `BLOB`, `CLOB`, and `NCLOB` is currently in incubating state, that is, the exact semantics, configuration options and so forth might change in future revisions, based on feedback we receive.
Use of the `BLOB`, `CLOB`, and `NCLOB` with the {prodname} Oracle connector is a Technology Preview feature only.
Technology Preview features are not supported with Red Hat production service level agreements (SLAs) and might not be functionally complete.
Red Hat does not recommend using them in production. These features provide early access to upcoming product features, enabling customers to test functionality and provide feedback during the development process.
For more information about the support scope of Red Hat Technology Preview features, see link:https://access.redhat.com/support/offerings/techpreview[https://access.redhat.com/support/offerings/techpreview].
|Either the raw bytes (the default), a base64-encoded String, or a hex-encoded String, based on the xref:#oracle-property-binary-handling-mode[`binary.handling.mode`] connector configuration property setting.
This means that change events will never contain the value of an unchanged `CLOB`, `NCLOB`, or `BLOB` column,
but a placeholder as defined by the connector property, `unavailable.value.placeholder`.
If the value of a `CLOB`, `NCLOB`, or `BLOB` column gets updated, the new value will be contained in the `after` part of the corresponding update change events whereas the unavailable value placeholder will be used in the `before` part.
|`org.apache.kafka.connect.data.Decimal` if using `BYTES` +
+
Handled equivalently to `NUMBER` (note that S defaults to 0 for `DECIMAL`).
|`DOUBLE PRECISION`
|`STRUCT`
|`io.debezium.data.VariableScaleDecimal` +
+
Contains a structure with two fields: `scale` of type `INT32` that contains the scale of the transferred value and `value` of type `BYTES` containing the original value in an unscaled form.
|`FLOAT[(P)]`
|`STRUCT`
|`io.debezium.data.VariableScaleDecimal` +
+
Contains a structure with two fields: `scale` of type `INT32` that contains the scale of the transferred value and `value` of type `BYTES` containing the original value in an unscaled form.
|`INTEGER`, `INT`
|`BYTES`
|`org.apache.kafka.connect.data.Decimal` +
+
`INTEGER` is mapped in Oracle to NUMBER(38,0) and hence can hold values larger than any of the `INT` types could store
Contains a structure with two fields: `scale` of type `INT32` that contains the scale of the transferred value and `value` of type `BYTES` containing the original value in an unscaled form.
Contains a structure with two fields: `scale` of type `INT32` that contains the scale of the transferred value and `value` of type `BYTES` containing the original value in an unscaled form.
Oracle does not natively have support for a `BOOLEAN` data type; however,
it is common practice to use other data types with certain semantics to simulate the concept of a logical `BOOLEAN` data type.
The operator can configure the out-of-the-box `NumberOneToBooleanConverter` custom converter that would either map all `NUMBER(1)` columns to a `BOOLEAN` or if the `selector` parameter is set,
then a subset of columns could be enumerated using a comma-separated list of regular expressions.
{prodname} connectors handle decimals according to the setting of the xref:{link-oracle-connector}#oracle-property-decimal-handling-mode[`decimal.handling.mode` connector configuration property].
When the `decimal.handling.mode` property is set to `precise`, the connector uses Kafka Connect `org.apache.kafka.connect.data.Decimal` logical type for all `DECIMAL` and `NUMERIC` columns.
When the `decimal.handling.mode` configuration property is set to `string`, the connector represents `DECIMAL`,`NUMERIC`, and `MONEY` values as their formatted string representation, and encodes the values as shown in the following table.
+
.Mappings when `decimal.handling.mode` is `string`
Other than Oracle's `INTERVAL`, `TIMESTAMP WITH TIME ZONE` and `TIMESTAMP WITH LOCAL TIME ZONE` data types, the other temporal types depend on the value of the `time.precision.mode` configuration property.
When the `time.precision.mode` configuration property is set to `adaptive` (the default), then the connector determines the literal and semantic type for the temporal types based on the column's data type definition so that events _exactly_ represent the values in the database:
The number of micro seconds for a time interval using the `365.25 / 12.0` formula for days per month average. +
+
`io.debezium.time.Interval` (when `interval.handling.mode` is set to `string`) +
+
The string representation of the interval value that follows the pattern `P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S`, for example, `P1Y2M3DT4H5M6.78S`.
The number of micro seconds for a time interval using the `365.25 / 12.0` formula for days per month average. +
+
`io.debezium.time.Interval` (when `interval.handling.mode` is set to `string`) +
+
The string representation of the interval value that follows the pattern `P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S`, for example, `P1Y2M3DT4H5M6.78S`.
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:
The number of micro seconds for a time interval using the `365.25 / 12.0` formula for days per month average. +
+
`io.debezium.time.Interval` (when `interval.handling.mode` is set to `string`) +
+
The string representation of the interval value that follows the pattern `P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S`, for example, `P1Y2M3DT4H5M6.78S`.
The number of micro seconds for a time interval using the `365.25 / 12.0` formula for days per month average. +
+
`io.debezium.time.Interval` (when `interval.handling.mode` is set to `string`) +
+
The string representation of the interval value that follows the pattern `P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S`, for example, `P1Y2M3DT4H5M6.78S`.
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:
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 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.
For information about using Vagrant to set up Oracle in a virtual machine, see the https://github.com/debezium/oracle-vagrant-box/[Debezium Vagrant Box for Oracle database] GitHub repository.
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.
[source,indent=0]
----
ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS;
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.
It is customary for a logical or physical standby to exist in the case of an Oracle production failure.
When a failure occurs and the standby instance is promoted to production, the database must be opened for read/write transactions before the {prodname} Oracle connector can connect to the database.
When using a physical standby, it is sufficient to reconfigure the {prodname} Oracle connector to use the hostname of the standby once the database is open.
In the case of a logical standby, the standby is not an exact copy of the production database, so the SCN offsets in the standby differ from those in the production database.
If you use a logical standby, to help ensure that {prodname} does not miss any change events, after the database is open, configure a new connector and perform a new database snapshot.
To deploy a {prodname} Oracle connector, you install the {prodname} Oracle connector archive, configure the connector, and start the connector by adding its configuration to Kafka Connect.
.Prerequisites
* link:https://zookeeper.apache.org/[Apache ZooKeeper], link:http://kafka.apache.org/[Apache Kafka], and link:{link-kafka-docs}.html#connect[Kafka Connect] are installed.
. xref:obtaining-the-oracle-jdbc-driver[Download the Oracle JDBC driver from Maven Central and extract it to the directory with the connector JAR files.]
* xref:{link-oracle-connector}#oracle-example-configuration[Configure the connector] and xref:{link-oracle-connector}#oracle-adding-connector-configuration[add the configuration to your Kafka Connect cluster.]
Due to licensing requirements, the {prodname} Oracle connector archive does not include the Oracle JDBC driver that the connector requires to connect to an Oracle database.
To enable the connector to access the database, you must add the driver to your connector environment.
For more information, see xref:obtaining-the-oracle-jdbc-driver[Obtaining the Oracle JDBC driver].
To deploy a {prodname} Oracle connector, you must build a custom Kafka Connect container image that contains the {prodname} connector archive, and then push this container image to a container registry.
You then need to create the following custom resources (CRs):
* A `KafkaConnect` CR that defines your Kafka Connect instance.
The `image` property in the CR specifies the name of the container image that you create to run your {prodname} connector.
You apply this CR to the OpenShift instance where link:https://access.redhat.com/products/red-hat-amq#streams[Red Hat {StreamsName}] is deployed.
{StreamsName} offers operators and images that bring Apache Kafka to OpenShift.
* A `KafkaConnector` CR that defines your {prodname} Oracle connector.
Apply this CR to the same OpenShift instance where you apply the `KafkaConnect` CR.
* Oracle Database is running and you completed the steps to {LinkDebeziumUserGuide}#setting-up-oracle-for-use-with-the-debezium-oracle-connector[set up Oracle to work with a {prodname} connector].
* {StreamsName} is deployed on OpenShift and is running Apache Kafka and Kafka Connect.
For more information, see link:{LinkDeployStreamsOpenShift}[{NameDeployStreamsOpenShift}]
* You have an account and permissions to create and manage containers in the container registry (such as `quay.io` or `docker.io`) to which you plan to add the container that will run your {prodname} connector.
.. Create a new {prodname} Oracle KafkaConnect custom resource (CR).
For example, create a KafkaConnect CR with the name `dbz-connect.yaml` that specifies `annotations` and `image` properties as shown in the following example:
+
[source,yaml,subs="+attributes"]
----
apiVersion: {KafkaConnectApiVersion}
kind: KafkaConnect
metadata:
name: my-connect-cluster
annotations:
strimzi.io/use-connector-resources: "true" // <1>
spec:
#...
image: debezium-container-for-oracle // <2>
----
<1> `metadata.annotations` indicates to the Cluster Operator that KafkaConnector resources are used to configure connectors in this Kafka Connect cluster.
This property overrides the `STRIMZI_DEFAULT_KAFKA_CONNECT_IMAGE` variable in the Cluster Operator
.. Apply the `KafkaConnect` CR to the OpenShift Kafka Connect environment by entering the following command:
+
[source,shell,options="nowrap"]
----
oc create -f dbz-connect.yaml
----
+
The command adds a Kafka Connect instance that specifies the name of the image that you created to run your {prodname} connector.
. Create a `KafkaConnector` custom resource that configures your {prodname} Oracle connector instance.
+
You configure a {prodname} Oracle connector in a `.yaml` file that specifies the configuration properties for the connector.
The connector configuration might instruct {prodname} to produce events for a subset of the schemas and tables, or it might set properties so that {prodname} ignores, masks, or truncates values in specified columns that are sensitive, too large, or not needed.
+
The following example configures a {prodname} connector that connects to an Oracle host IP address, on port `1521`.
This host has a database named `ORCLCDB`, and `server1` is the server's logical name.
|Logical name that identifies and provides a namespace for the Oracle database server from which the connector captures changes.
|10
|The list of Kafka brokers that this connector uses to write and recover DDL statements to the database history topic.
|11
|The name of the database history topic where the connector writes and recovers DDL statements. This topic is for internal use only and should not be used by consumers.
|===
. Create your connector instance with Kafka Connect.
For example, if you saved your `KafkaConnector` resource in the `inventory-connector.yaml` file, you would run the following command:
+
[source,shell,options="nowrap"]
----
oc apply -f inventory-connector.yaml
----
+
The preceding command registers `inventory-connector` and the connector starts to run against the `server1` database as defined in the `KafkaConnector` CR.
There are two methods for obtaining the driver, depending on the deployment method that you use.
* If you xref:openshift-streams-oracle-connector-deployment[use {StreamsName} to add the connector to your Kafka Connect image], add an artifact reference to the `KafkaConnect` custom resource and then add the location of the artifact as the `url` value.
* If you xref:deploying-debezium-oracle-connectors[use a Dockerfile to build the connector], download the required driver file directly from Oracle and add it to your Kafka Connect environment.
.Procedure
. Complete one of the following procedures, depending on your deployment type:
** If you use {StreamsName} to deploy the connector:
.. In the YAML for the `KafkaConnector` custom resource (CR), add the following URL path for the driver to the `artifacts.url` field for the `debezium-connector-oracle` artifact:
For more information about the YAML file for the `KafkaConnector` CR, see xref:openshift-streams-oracle-connector-deployment[Using {StreamsName} to deploy a {prodname} Oracle connector].
** If you use a Dockerfile to deploy the connector:
.. From a browser, link:https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/{ojdbc8-version}/ojdbc8-{ojdbc8-version}.jar[download the 'ojdbc8.jar' from Maven Central].
.. Copy the downloaded driver file to the directory that contains the {prodname} Oracle connector JAR file (that is, `debezium-connector-oracle-{debezium-version}.jar`).
. From a browser, link:https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/{ojdbc8-version}/ojdbc8-{ojdbc8-version}.jar[download the 'ojdbc8.jar' from Maven Central].
. Copy the downloaded driver file to the directory that contains the {prodname} Oracle connector JAR file (`debezium-connector-oracle-{debezium-version}.jar`).
<11> The list of Kafka brokers that this connector uses to write and recover DDL statements to the database history topic.
<12> The name of the database history topic where the connector writes and recovers DDL statements. This topic is for internal use only and should not be used by consumers.
"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)))",
When you configure a {prodname} Oracle connector for use with an Oracle CDB, you must specify a value for the property `database.pdb.name`, which names the PDB that you want the connector to capture changes from.
For non-CDB installation, do *not* specify the `database.pdb.name` property.
====
.Example: {prodname} Oracle connector configuration for non-CDB deployments
For the complete list of the configuration properties that you can set for the {prodname} Oracle connector, see xref:{link-oracle-connector}#oracle-connector-properties[Oracle connector properties].
After the connector starts, it xref:{link-oracle-connector}#oracle-snapshots[performs a consistent snapshot] of the Oracle databases that the connector is configured for.
* xref:debezium-oracle-connector-database-history-configuration-properties[Database history connector configuration properties] that control how {prodname} processes events that it reads from the database history topic.
** xref:oracle-pass-through-database-history-properties-for-configuring-producer-and-consumer-clients[Pass-through database history properties]
* xref:debezium-oracle-connector-pass-through-database-driver-configuration-properties[Pass-through database driver properties] that control the behavior of the database driver.
|Unique name for the connector. Attempting to register again with the same name will fail. (This property is required by all Kafka Connect connectors.)
|The maximum number of tasks that should be created for this connector. The Oracle connector always uses a single task and therefore does not use this value, so the default is always acceptable.
If you change the name value, after a restart, instead of continuing to emit events to the original topics, the connector emits subsequent events to topics whose names are based on the new value.
The connector is also unable to recover its database history topic.
`schema_only_recovery`:: This is a recovery setting for a connector that has already been capturing changes.
When you restart the connector, this setting enables recovery of a corrupted or lost database history topic.
You might set it periodically to "clean up" a database history topic that has been growing unexpectedly.
Database history topics require infinite retention.
Note this mode is only safe to be used when it is guaranteed that no schema changes happened since the point in time the connector was shut down before and the point in time the snapshot is taken.
After the snapshot is complete, the connector continues to read change events from the database's redo logs except when `snapshot.mode` is configured as `initial_only`.
a|Controls whether and for how long the connector holds a table lock. Table locks prevent certain types of changes table operations from occurring while the connector performs a snapshot.
You can set the following values:
`shared`:: Enables concurrent access to the table, but prevents any session from acquiring an exclusive table lock.
The connector acquires a `ROW SHARE` level lock while it captures table schema.
`none`:: Prevents the connector from acquiring any table locks during the snapshot.
Use this setting only if no schema changes might occur during the creation of the snapshot.
|An optional, comma-separated list of regular expressions that match the fully-qualified names (`_<schemaName>_._<tableName>_`) of the tables to include in a snapshot.
The specified items must be named in the connector's xref:{context}-property-table-include-list[`table.include.list`] property.
This property takes effect only if the connector's `snapshot.mode` property is set to a value other than `never`. +
For each table in the list, add a further configuration property that specifies the `SELECT` statement for the connector to run on the table when it takes a snapshot.
The specified `SELECT` statement determines the subset of table rows to include in the snapshot.
Use the following format to specify the name of this `SELECT` statement property: +
From a `customers.orders` table that includes the soft-delete column, `delete_flag`, add the following properties if you want a snapshot to include only those records that are not soft-deleted:
|An optional, comma-separated list of regular expressions that match names of schemas for which you *want* to capture changes. Any schema name not included in `schema.include.list` is excluded from having its changes captured. By default, all non-system schemas have their changes captured. Do not also set the `schema.exclude.list` property.
In environments that use the LogMiner implementation, you must use POSIX regular expressions only.
|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.
|An optional, comma-separated list of regular expressions that match names of schemas for which you *do not* want to capture changes. Any schema whose name is not included in `schema.exclude.list` has its changes captured, with the exception of system schemas. Do not also set the `schema.include.list` property.
In environments that use the LogMiner implementation, you must use POSIX regular expressions only.
|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.
|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.
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 {link-java7-standard-names}[MessageDigest section] of the Java Cryptography Architecture Standard Algorithm Name Documentation. +
+
In the following example, `CzQMA0cB5K` is a randomly selected salt. +
|Specifies how binary (`blob`) columns should be represented in change events, including: `bytes` represents binary data as byte array (default), `base64` represents binary data as base64-encoded String, `hex` represents binary data as hex-encoded (base16) String
| Specifies how the connector should handle values for `interval` columns: +
+
`numeric` represents intervals using approximate number of microseconds. +
+
`string` represents intervals exactly by using the string pattern representation `P<years>Y<months>M<days>DT<hours>H<minutes>M<seconds>S`. For example: `P1Y2M3DT4H5M6.78S`.
If xref:oracle-property-max-queue-size[`max.queue.size`] is also set, writing to the queue is blocked when the size of the queue reaches the limit specified by either property.
For example, if you set `max.queue.size=1000`, and `max.queue.size.in.bytes=5000`, writing to the queue is blocked after the queue contains 1000 records, or after the volume of the records in the queue reaches 5000 bytes.
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 {link-kafka-docs}/#compaction[log compaction] enabled.
|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, {prodname} uses the primary key column of a table as the message key for records that it emits.
In place of the default, or to specify a key for tables that lack a primary key, you can configure custom message keys based on one or more columns. +
To establish a custom message key for a table, list the table, followed by the columns to use as the message key.
|An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns to be truncated in change event messages if their length exceeds the specified number of characters.
Length is specified as a positive integer. A configuration can include multiple properties that specify different lengths.
|An optional comma-separated list of regular expressions for masking column names in change event messages by replacing characters with asterisks (`*`). +
Specify the number of characters to replace in the name of the property, for example, `column.mask.with.8.chars`. +
|An optional comma-separated list of regular expressions that match the fully-qualified names of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change messages.
The schema parameters `pass:[_]pass:[_]debezium.source.column.type`, `pass:[_]pass:[_]debezium.source.column.length`, and `pass:[_]pass:[_]debezium.source.column.scale` are used to propagate the original type name and length (for variable-width types), respectively.
|An optional comma-separated list of regular expressions that match the database-specific data type name of columns whose original type and length should be added as a parameter to the corresponding field schemas in the emitted change messages.
The schema parameters `pass:[_]pass:[_]debezium.source.column.type`, `pass:[_]pass:[_]debezium.source.column.length` and `pass:[_]pass:[_]debezium.source.column.scale` are used to propagate the original type name and length (for variable-width types), respectively.
In such a a case, although the connector continues to read the redo log, it emits no change event messages, so that the offset in the Kafka topic remains unchanged.
Because the connector does not flush the latest system change number (SCN) that it read from the database, the database might retain the redo log files for longer than necessary.
|`true` when the connector configuration explicitly specifies the `key.converter` or `value.converter` parameters to use Avro, otherwise defaults to `false`.
|Specifies whether field names are normalized to comply with Avro naming requirements.
|Set the property to `true` if you want {prodname} to generate events with transaction boundaries and enriches data events envelope with transaction metadata.
|Controls the name of the topic to which the connector sends transaction metadata messages. The placeholder `${database.server.name}` can be used for referring to the connector's logical name; defaults to `${database.server.name}.transaction`, for example `dbserver1.transaction`.
|Specifies the mining strategy that controls how Oracle LogMiner builds and uses a given data dictionary for resolving table and column ids to names. +
|The minimum SCN interval size that this connector attempts to read from redo/archive logs. Active batch size is also increased/decreased by this amount for tuning connector throughput when needed.
|The 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.
Redo logs use a circular buffer that can be archived at any point.
In environments where online redo logs are archived frequently, this can lead to LogMiner session failures.
In contrast to redo logs, archive logs are guaranteed to be reliable.
Set this option to `true` to force the connector to mine archive logs only.
After you set the connector to mine only the archive logs, the latency between an operation being committed and the connector emitting an associated change event might increase.
The degree of latency depends on how frequently the database is configured to archive online redo logs.
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.
we have plans to enhance this behavior in a future release by means of adding a scalable transaction buffer, (see xref:{jira-url}/browse/DBZ-3123[DBZ-3123]).
|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 xref:oracle-property-log-mining-scn-gap-detection-time-interval-max-ms[`log.mining.scn.gap.detection.time.interval.max.ms`] then an SCN gap is detected, and the connector uses a mining window larger than the configured maximum batch.
|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 xref:oracle-property-log-mining-scn-gap-detection-gap-size-min[`log.mining.scn.gap.detection.gap.size.min`], then an SCN gap is detected and the connector uses a mining window larger than the configured maximum batch.
This field is required to enable Oracle RAC support.
Specify the list of RAC nodes by using one of the following methods:
* Specify a value for xref:oracle-property-database-port[`database.port`], and use the specified port value for each address in the `rac.nodes` list.
For example:
+
[source,properties]
----
database.port=1521
rac.nodes=192.168.1.100,192.168.1.101
----
* Specify a value for xref:oracle-property-database-port[`database.port`], and override the default port for one or more entries in the list.
The list can include entries that use the default `database.port` value, and entries that define their own unique port values.
For example:
+
[source,properties]
----
database.port=1521
rac.nodes=192.168.1.100,192.168.1.101:1522
----
If you supply a raw JDBC URL for the database by using the xref:oracle-property-database-url[`database.url`] property, instead of defining a value for `database.port`, each RAC node entry must explicitly specify a port value.
a|Fully-qualified name of the data collection that is used to send xref:{link-signalling}#debezium-signaling-enabling-signaling[signals] to the connector. +
The {prodname} 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.
|The number of times the system change number has been checked for advancement and remains unchanged.
This is an indicator that long-running transaction(s) are ongoing and preventing the connector from flushing the latest processed system change number to the connector's offsets.
Under optimal operations, this should always be or remain close to `0`.
|The number of DDL records that have been detected but could not be parsed by the DDL parser.
This should always be `0`; however when allowing unparsable DDL to be skipped, this metric can be used to determine if any warnings have been written to the connector logs.
By default, the connector stops when it encounters a DDL statement that it cannot parse.
You can use {prodname} link:/documentation/reference/configuration/signalling[signaling] to trigger the update of the database schema from such DDL statements.
After the `schema-changes` signal is inserted, the connector must be restarted with an altered configuration that includes specifying the <<{context}-property-database-history-skip-unparseable-ddl, `+database.history.skip.unparseable.ddl+`>> option as `true`.
After the connector's commit SCN advances beyond the DDL change, to prevent unparseable DDL statements from being skipped unexpectedly, return the connector configuration to its previous state.
The connector can be toggled to use Oracle XStream instead.
To configure the connector to use Oracle XStream, you must apply specific database and connector configurations that differ from those that you use with LogMiner.
.Prerequisites
* To use the XStream API, you must have a license for the GoldenGate product.
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.
[source,indent=0]
----
ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS;
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.
=== Obtaining the Oracle JDBC driver and XStream API files
The {prodname} Oracle connector requires the Oracle JDBC driver (`ojdbc8.jar`) to connect to Oracle databases.
If the connector uses XStream to access the database, you must also have the XStream API (`xstreams.jar`).
Licensing requirements prohibit {prodname} from including these files in the Oracle connector archive.
However, the required files are available for free download as part of the Oracle Instant Client.
The following steps describe how to download the Oracle Instant Client and extract the required files.
.Procedure
. From a browser, download the https://www.oracle.com/database/technologies/instant-client/downloads.html[Oracle Instant Client package] for your operating system.
. Extract the archive, and then open the `instantclient___<version>__` directory.
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 https://issues.redhat.com/browse/DBZ-4741[DBZ-4741] and will be addressed in a future release.
After the connector examines the redo and archive logs, if it cannot find the SCN that is recorded in the connector offsets, it returns the preceding error.
Because the connector uses the SCN to determine where to resume processing, if the expected SCN if not found, a new snapshot must be completed.
To be available for mining, a record must include a filename in the `NAME` column, a value of `NO` in the `DELETED` column, and a value of `A` (available) in the `STATUS` column.
If a record does not match any of these criteria, it is considered incomplete and cannot be mined.
This error means that the connector has attempted to execute an operation that must be executed against the parent index-organized table that contains the specified overflow table.
The connector's `table.include.list` or `table.exclude.list` configuration options should then be adjusted to explicitly include or exclude the appropriate tables to avoid the connector from attempting to capture changes from the child index-organized table.
=== LogMiner adapter does not capture changes made by SYS or SYSTEM
Oracle uses the `SYS` and `SYSTEM` accounts for lots of internal changes and therefore the connector automatically filters changes made by these users when fetching changes from LogMiner.
=== Connector stops capturing changes from Oracle on AWS
Due to the https://aws.amazon.com/blogs/networking-and-content-delivery/best-practices-for-deploying-gateway-load-balancer[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.
ifdef::product[]
To prevent timeouts with the AWS Gateway Load Balancer, enable keep-alive packets from the Kafka Connect environment, by performing the following task as root or a super-user:
endif::product[]
ifdef::community[]
To prevent timeouts with the AWS Gateway Load Balancer, enable keep-alive packets from the kafka Connect or Debezium Server environment, by performing the following task as root or a super-user in the environment that hosts the connector:
endif::community[]
From a terminal, run the following command:
```shell
sysctl -w net.ipv4.tcp_keepalive_time=60
```
Edit `/etc/sysctl.conf` and set the value of the following variable as shown:
```properties
net.ipv4.tcp_keepalive_time=60
```
Reconfigure the {prodname} for Oracle connector to use the `database.url` property rather than `database.hostname` and add the `(ENABLE=broken)` Oracle connect string descriptor as shown in the following example:
The precdeding 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.