This connector is currently in incubating state, i.e. exact semantics, configuration options etc. may change in future revisions, based on the feedback we receive. Please let us know if you encounter any problems.
Most notably, the connector does not yet support changes to the structure of captured tables (e.g. `ALTER TABLE...`) after the initial snapshot has been completed
{prodname} ingests change events from Oracle using the https://docs.oracle.com/database/121/XSTRM/xstrm_intro.htm#XSTRM72647[XStream API] or directly by LogMiner.
This assumes the multi-tenancy configuration (with a container database and at least one pluggable database);
if you're not using this model, adjust the steps accordingly.
You can find a template for setting up Oracle in a virtual machine (via Vagrant) in the https://github.com/debezium/oracle-vagrant-box/[oracle-vagrant-box/] repository.
Enable GoldenGate replication and archive log mode:
[source,indent=0]
----
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;
----
Furthermore, in order to capture the _before_ state of changed rows, supplemental logging must be enabled for the captured tables or the database in general.
E.g. like so for a specific table:
[source,indent=0]
----
ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS;
It's recommended to have a minimum of `5` redo log groups each with `2` redo files per group.
Please refer to the Oracle documentation on how to manipulate redo log groups and files for your database version.
====
Enable archive log mode:
[source,indent=0]
----
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/oradta/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;
----
Furthermore, in order to capture the _before_ state of changed rows, supplemental logging must be enabled for the captured tables or the database in general.
E.g. like so for a specific table:
[source,indent=0]
----
ALTER TABLE inventory.customers ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS;
So, by default (snapshotting mode *initial*) the connector will upon first startup perform an initial _consistent snapshot_ of the database
(meaning the structure and data within any tables to be captured as per the connector's filter configuration).
Each snapshot consists of the following steps:
1. Determine the tables to be captured
2. Obtain an `IN EXCLUSIVE MODE` lock on each of the monitored tables to ensure that no structural changes can occur to any of the tables.
3. Read the current SCN ("system change number") position in the server's redo log.
4. Capture the structure of all relevant tables.
5. Release the locks obtained in step 2, i.e. the locks are held only for a short period of time.
6. Scan all of the relevant database tables and schemas as valid at the SCN position read in step 3 (`SELECT * FROM ... AS OF SCN 123`), and generate a `READ` event for each row and write that event to the appropriate table-specific Kafka topic.
7. Record the successful completion of the snapshot in the connector offsets.
If the connector fails, is rebalanced, or stops after step 1 begins but before step 7 completes,
upon restart the connector will begin a new snapshot.
Once the Oracle connector does complete its initial snapshot, it continues streaming from the position read during step 3,
ensuring that it does not miss any updates that occurred while the snapshot was taken.
If the connector stops again for any reason, upon restart it will simply continue streaming changes from where it previously left off.
In this case, step 6 from the snapshotting routine described above is not applied.
In other words, the connector still captures the structure of the relevant tables, but it does not create any `READ` events representing the complete dataset at the point of connector start-up.
This can be useful if you are interested in data changes only from now onwards but not the complete current state of all records.
{prodname} emits a new message to this topic whenever a new table is streamed from or when the structure of the table is altered ({link-prefix}:{link-oracle-connector}#oracle-schema-evolution[schema evolution procedure must be followed]).
All data change events produced by the Oracle connector have a key and a value, although the structure of the key and value depend on the table from which the change events originated (see {link-prefix}:{link-oracle-connector}#oracle-topic-names[Topic names]).
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 Latin letters or an underscore (e.g., [a-z,A-Z,\_]),
and the remaining characters in the logical server name and all characters in the schema and table names must be Latin letters, digits, or an underscore (e.g., [a-z,A-Z,0-9,\_]).
If not, then all invalid characters will automatically be replaced with an underscore character.
This can lead to unexpected conflicts when the logical server name, schema names, and table names contain other characters, and the only distinguishing characters between table full names are invalid and thus replaced with underscores.
For a given table, the change event's key will have a structure that contains a field for each column in the primary key (or unique key constraint) of the table at the time the event was created.
Consider a `customers` table defined in the `inventory` database schema:
[source,sql,indent=0]
----
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 `database.server.name` configuration property has the value `server1`,
every change event for the `customers` table while it has this definition will feature the same key structure, which in JSON looks like this:
[source,json,indent=0]
----
{
"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 describing what is in the key portion, and in our case that means that the `payload` value is not optional, is a structure defined by a schema named `server1.DEBEZIUM.CUSTOMERS.Key`, and has one required field named `id` of type `int32`.
If you look at the value of the key's `payload` field, you can see that it is indeed a structure (which in JSON is just an object) with a single `id` field, whose value is `1004`.
Therefore, you can interpret this key as describing the row in the `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 will be null. This makes sense since the rows in a table without a primary or unique key constraint cannot be uniquely identified.
Like the message key, the value of a change event message has a _schema_ section and _payload_ section.
The payload section of every change event value produced by the Oracle connector has an _envelope_ structure with the following fields:
* `op` is 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` is an optional field that if present contains the state of the row _before_ the event occurred. The structure will be described by the `server1.INVENTORY.CUSTOMERS.Value` Kafka Connect schema, which the `server1` connector uses for all rows in the `inventory.customers` table.
[WARNING]
====
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` is 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` is 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).
[TIP]
====
The `commit_scn` field is optional and describes the SCN of the transaction commit that the change event participates within.
This field is only present when using the LogMiner connection adapter.
* `ts_ms` is optional and if present contains the time (using the system clock in the JVM running the Kafka Connect task) at which the connector processed the event.
And of course, the _schema_ portion of the event message's value contains a schema that describes this envelope structure and the nested fields within it.
If we look at the `schema` portion of this event's _value_, we can see the schema for the _envelope_, the schema for the `source` structure (which is specific to the Oracle connector and reused across all events), and the table-specific schemas for the `before` and `after` fields.
[TIP]
====
The names of the schemas for the `before` and `after` fields are of the form _logicalName_._schemaName_._tableName_.Value, and thus are entirely independent from all other schemas for all other tables.
This means that when using the link:/docs/faq/#avro-converter[Avro Converter], the resulting Avro schems for _each table_ in each _logical source_ have their own evolution and history.
====
If we look at the `payload` portion of this event's _value_, we can see the information in the event, namely that it is describing that the row was created (since `op=c`), and that the `after` field value contains the values of the new inserted row's' `ID`, `FIRST_NAME`, `LAST_NAME`, and `EMAIL` columns.
[TIP]
====
It may appear that the JSON representations of the events are much larger than the rows they describe.
This is true, because the JSON representation must include the _schema_ and the _payload_ portions of the message.
It is possible and even recommended to use the link:/docs/faq/#avro-converter[Avro Converter] to dramatically decrease the size of the actual messages written to the Kafka topics.
The value of an _update_ change event on this table will actually have the exact same _schema_, and its payload will be structured the same but will hold different values.
When we compare this to the value in the _insert_ event, we see a couple of differences in the `payload` section:
* The `op` field value is now `u`, signifying that this row changed because of an update
* The `before` field now has the state of the row with the values before the database commit
* The `after` field now has the updated state of the row, and here was can see that the `EMAIL` value is now `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.
There are several things we can learn by just looking at this `payload` section. We can compare the `before` and `after` structures to determine what actually changed in this row because of the commit.
The `source` structure tells us information about Oracle's record of this change (providing traceability), but more importantly this has information we can compare to other events in this and other topics to know whether this event occurred before, after, or as part of the same Oracle commit as other events.
When the columns for a row's primary/unique key are updated, the value of the row's key has changed so {prodname} will output _three_ events: a `DELETE` event and a {link-prefix}:{link-oracle-connector}#oracle-tombstone-events[tombstone event] with the old key for the row, followed by an `INSERT` event with the new key for the row.
So far we've seen samples of _create_ and _update_ events.
Now, let's look at the value of a _delete_ event for the same table. Once again, the `schema` portion of the value will be exactly the same as with the _create_ and _update_ events:
When a row is deleted, the _delete_ event value listed above still works with log compaction, since Kafka can still remove all earlier messages with that same key.
But only if the message value is `null` will Kafka know that it can remove _all messages_ with that same key.
To make this possible, {prodname}'s Oracle connector always follows the _delete_ event with a special _tombstone_ event that has the same key but `null` value.
* `id` - string representation of unique transaction identifier
* `event_count` (for `END` events) - total number of events emmitted by the transaction
* `data_collections` (for `END` events) - an array of pairs of `data_collection` and `event_count` that provides number of events emitted by changes originating from given data collection
As described above, the {prodname} Oracle connector represents the changes to rows with events that are structured like the table in which the row exist.
The event contains a field for each column value, and how that value is represented in the event depends on the Oracle data type of the column.
This section describes this mapping from Oracle's data types to a _literal type_ and _semantic type_ within the events' fields.
Here, the _literal type_ describes how the value is literally represented using Kafka Connect schema types, namely `INT8`, `INT16`, `INT32`, `INT64`, `FLOAT32`, `FLOAT64`, `BOOLEAN`, `STRING`, `BYTES`, `ARRAY`, `MAP`, and `STRUCT`.
The _semantic type_ describes how the Kafka Connect schema captures the _meaning_ of the field using the name of the Kafka Connect schema for the field.
Support for further data types will be added in subsequent releases.
|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.
|`NUMBER(P, S > 0)`
|`BYTES`
|`org.apache.kafka.connect.data.Decimal`
|
|`NUMBER(P, S <= 0)`
|`INT8` / `INT16` / `INT32` / `INT64`
|n/a
|`NUMBER` columns with a scale of 0 represent integer numbers; a negative scale indicates rounding in Oracle, e.g. a scale of -2 will cause rounding to hundreds. +
Depending on the precision and scale, a matching Kafka Connect integer type will be chosen: `INT8` if P - S < 3, `INT16` if P - S < 5, `INT32` if P - S < 10 and `INT64` if P - S < 19. +
If P - S >= 19, the column will be mapped to `BYTES` (`org.apache.kafka.connect{zwsp}.data.Decimal`).
|`SMALLINT`
|`BYTES`
|`org.apache.kafka.connect.data.Decimal`
|`SMALLINT` is mapped in Oracle to NUMBER(38,0) and hence can hold values larger than any of the `INT` types could store
|`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
|`NUMERIC[(P, S)]`
|`BYTES` / `INT8` / `INT16` / `INT32` / `INT64`
|`org.apache.kafka.connect.data.Decimal` if using `BYTES`
|Handled equivalently to `NUMBER` (note that S defaults to 0 for `NUMERIC`).
|`DECIMAL[(P, S)]`
|`BYTES` / `INT8` / `INT16` / `INT32` / `INT64`
|`org.apache.kafka.connect.data.Decimal` if using `BYTES`
|Handled equivalently to `NUMBER` (note that S defaults to 0 for `DECIMAL`).
|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.
|Contains a structure with two fields: `scale` of type `INT32` that contains the scale of the transferred value and `value` of type `BYTES` containing the original value in an unscaled form.
When `decimal.handling.mode` configuration property is set to `precise`, then the connector will use the predefined Kafka Connect `org.apache.kafka.connect.data.Decimal` or `io.debezium.data.VariableScaleDecimal` logical types for numeric columns as described above.
This is the default mode.
However, when `decimal.handling.mode` configuration property is set to `double`, then the connector will represent the values as Java double values with schema type `FLOAT64`.
The last option for `decimal.handling.mode` configuration property is `string`. In this case the connector will represent the values as their formatted string representation with schema type `STRING`.
The following example configuration illustrates that by adding the `database.connection.adapter`, the connector can be toggled to use the LogMiner implementation.
We do encourage the use of the LogMiner implementation for testing purposes and providing us with your feedback, but we do not yet recommend its use in production as its still under active development.
The {prodname} Oracle connector has three metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.
* <<snapshot-metrics, snapshot metrics>>; for monitoring the connector when performing snapshots
Please refer to the {link-prefix}:{link-debezium-monitoring}#monitoring-debezium[monitoring documentation] for details of how to expose these metrics via JMX.
|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.
|Logical name that identifies and provides a namespace for the particular Oracle database server being monitored. The logical name should be unique across all other connectors, since it is used as a prefix for all Kafka topic names emanating from this connector.
|A list of host/port pairs that the connector will use for establishing an initial connection to the Kafka cluster. This connection will be used for retrieving database schema history previously stored by the connector, and for writing each DDL statement read from the source database. This should point to the same Kafka cluster used by the Kafka Connect process.
|A mode for taking an initial snapshot of the structure and optionally data of captured tables. Supported values are _initial_ (will take a snapshot of structure and data of captured tables; useful if topics should be populated with a complete representation of the data from the captured tables) and _schema_only_ (will take a snapshot of the structure of captured tables only; useful if only changes happening from now onwards should be propagated to topics). Once the snapshot is complete, the connector will continue reading change events from the database's redo logs.
|An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be monitored; any table not included in the include list will be excluded from monitoring. Each identifier is of the form _schemaName_._tableName_. By default the connector will monitor every non-system table in each monitored database. May not be used with `table.exclude.list`.
|An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be excluded from monitoring; any table not included in the exclude list will be monitored. Each identifier is of the form _schemaName_._tableName_. May not be used with `table.include.list`.
|An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be pseudonyms in the change event message values with a field value consisting of the hashed value using the algorithm `_hashAlgorithm_` and salt `_salt_`.
Based on the used hash function referential integrity is kept while data is pseudonymized. Supported hash functions are described in the {link-java7-standard-names}[MessageDigest section] of the Java Cryptography Architecture Standard Algorithm Name Documentation.
The hash is automatically shortened to the length of the column.
Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer or zero. Fully-qualified names for columns are of the form _pdbName_._schemaName_._tableName_._columnName_.
Note: Depending on the `_hashAlgorithm_` used, the `_salt_` selected and the actual data set, the resulting masked data set may not be completely anonymized.
| Specifies how the connector should handle floating point values for `NUMBER`, `DECIMAL` and `NUMERIC` columns: `precise` (the default) represents them precisely using `java.math.BigDecimal` values represented in change events in a binary form; or `double` represents them using `double` values, which may result in a loss of precision but will be far easier to use. `string` option encodes values as formatted string which is easy to consume but a semantic information about the real type is lost. See <<decimal-values>>.
|Positive integer value that specifies the maximum size of the blocking queue into which change events read from the database log are placed before they are written to Kafka. This queue can provide backpressure to the binlog reader when, for example, writes to Kafka are slower or if Kafka is not available. Events that appear in the queue are not included in the offsets periodically recorded by this connector. Defaults to 8192, and should always be larger than the maximum batch size specified in the `max.batch.size` property.
|Positive integer value that specifies the maximum size of each batch of events that should be processed during each iteration of this connector. Defaults to 2048.
|Long value for the maximum size in bytes of the blocking queue. The feature is disabled by default, it will be active if it's set with 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. Defaults to 1000 milliseconds, or 1 second.
| Controls whether a tombstone event should be generated after a delete event. +
When `true` the delete operations are represented by a delete event and a subsequent tombstone event. When `false` only a delete event is sent. +
Emitting the tombstone event (the default behavior) allows Kafka to completely delete all events pertaining to the given key once the source record got deleted.
|An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be truncated in the change event message values if the field values are longer than the specified number of characters. Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer. Fully-qualified names for columns are of the form _pdbName_._schemaName_._tableName_._columnName_.
|An optional comma-separated list of regular expressions that match the fully-qualified names of character-based columns whose values should be replaced in the change event message values with a field value consisting of the specified number of asterisk (`*`) characters. Multiple properties with different lengths can be used in a single configuration, although in each the length must be a positive integer or zero. Fully-qualified names for columns are of the form _pdbName_._schemaName_._tableName_._columnName_.
|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` will be used to propagate the original type name and length (for variable-width types), respectively.
Useful to properly size corresponding columns in sink databases.
|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` will be used to propagate the original type name and length (for variable-width types), respectively.
Useful to properly size corresponding columns in sink databases.
|`true` when connector configuration explicitly specifies the `key.converter` or `value.converter` parameters to use Avro, otherwise defaults to `false`.