Debezium Connector for Cassandra
Overview
Cassandra is an open-sourced NoSQL database. Similar to most databases, the write path of Cassandra starts with the immeidate logging of a change into its commit log. The commit log resides locally on each node, recording every write made to that node.
Since Cassandra 3.0, a change data capture (CDC) feature is introduced. The CDC feature can be enabled on the table level by setting the table property cdc=true
, after which any commit log containing data for a CDC-enabled table will be moved to the CDC directory specified in cassandra.yaml
on disgard.
The Cassandra connector resides on each Cassandra node and monitors the cdc_raw
directory for change. It processes all local commit log segments as they are detected, produces a change event for every row-level insert, update, and delete operations in the commit log, publishes all change events for each table in a separate Kafka topic, and finally deletes the commit log from the cdc_raw
directory. This last step is important because once CDC is enabled, Cassandra itself cannot purge the commit logs. If the cdc_free_space_in_mb
fills up, writes to CDC-enabled tables will be rejected.
The connector is tolerant of failures. As the connector reads commit logs and produces events, it records each commit log segment’s filename and position along with each event. If the connector stops for any reason (including communication failures, network problems, or crashes), upon restart it simply continues reading the commit log where it last left off. This includes snapshots: if the snapshot was not completed when the connector is stopped, upon restart it will begin a new snapshot. We’ll talk later about how the connector behaves when things go wrong.
Cassandra is different from the other Debezium connectors since it is not implemented on top of the Kafka Connect framework. Instead it is a single JVM process that is intended to reside on each Cassandra node and publishes events to Kafka via a Kafka producer. |
The following features are currently not supported by the Cassandra connector. Changes resulted from any of these features are ignored:
|
Setting Up Cassandra
Before the Debezium Cassandra connector can be used to monitor the changes in a Cassandra cluster, CDC must be enabled on the node level and table level.
Enabling CDC on Node
To enable CDC, update the following CDC config in cassandra.yaml
:
cdc_enabled: true
Additional CDC configs are have the following default values:
cdc_raw_directory: $CASSANDRA_HOME/data/cdc_raw
cdc_free_space_in_mb: 4096
cdc_free_space_check_interval_ms: 250
cdc_enabled
enables or disables CDC operations node-widecdc_raw_directory
determines the destination for commit log segments to be moved after all corresponding memtables are flushedcdc_free_space_in_mb
is the maximum capacity allocated to store commit log segments, and defaults to the minimum of 4096 MB and 1/8 of volume space.cdc_free_space_check_interval_ms
is frequency with which we re-calculate the space taken up bycdc_raw_directory
to prevent burning CPU cycles unnecessarily when at capacity.
Enabling CDC on Table
Once CDC is enabled on the Cassandra node, each table must be be explicitly enabled for CDC as well via the CREATE TABLE or ALTER TABLE command. For example:
CREATE TABLE foo (a int, b text, PRIMARY KEY(a)) WITH cdc=true;
ALTER TABLE foo WITH cdc=true;
How the Cassandra Connector Works
This section goes into detail about how the Cassandra connector performs snapshots, transforms commit log events into Debezium change events, handles commit log life cycle, records events into Kafka, manages schema evolution, and behaves when things go wrong.
Snapshots
When the Cassandra connector first starts on a Cassandra node, it will by default perform an initial snapshot of the cluster. This is the default mode, since most of the time CDC is enabled on non-empty tables and commit logs do not contain the complete history.
The snapshot reader issues a SELECT statement to query all the columns in a table. Cassandra allows consistency level to be set either globally or on the statement level. For snapshotting, the consistency level is set on the statement level to ALL
by default to provide the highest consistency. This implies if one node goes down during the snapshot, the snapshot would not be able to continue and a subsequent re-snapshot is required once the node has been brought back online. You can adjust the consistency level of the snapshot to a lower consistency level in order to increase availability, provided that you understand the tradeoff with consistency.
In Cassandra 3.X, it is not possible to read strictly from the local Cassandra node. Starting in Cassandra 4.0, a |
Unlike relational databases, there is no read lock applied during a snapshot, so writes to Cassandra are not blocked during that snapshot. If the queried data has been modified by another client during the snapshot, those changes may be reflected in the snapshot result set.
If the connector fails or stops before the snapshot is completed, the connector will begin a new snapshot upon restarts. In the default snapshot mode (initial
), once the connector completes its initial snapshot, it will no longer perform any additional snapshots. The only exception would be during a connector restart: if cdc is enabled on a table, and then the connector is restarted, that table would be snapshotted.
The second snapshot mode (always
) allows the connetor to perform snapshot whenever necessary. it will check periodically for newly cdc-enabled tables, and snapshot them as soon as they are detected.
The third snapshot mode (‘never’) ensures the connector never performs snapshots. When a new connector is configured this way, it will only read the commit log in the CDC directory. This is not the default behavior because starting a new connector in this mode (without a snapshot) requires the commit logs to contain the entire history of all cdc-enabled tables, which is often not the case. Another use case for this mode is if there is one connector already doing the snapshotting, you can disable snapshot on others to avoid duplicated work.
Reading the Commit Log
The Cassandra connector will typically spend the vast majority of its time reading local commit logs on the Cassandra node.
Commit logs’ binary data are deserialized with Cassandra’s CommitLogReader and CommitLogReadHandler. Each deserialized object is called a mutation
in Cassandra. A mutation
contains one or more change events.
As the Cassandra connector reads the commit log, it transform the log events into Debezium create, update, or delete events that include the position in the commit log where the event was found. The Cassandra connector encode these change events with Kafka Connect converters and publish them to the appropriate Kafka topics.
Limitations of Commit Logs
Cassandra’s commit logs come with a set of limitations, which are critical for interpreting CDC events correctly:
Commit logs only arrive in
cdc_raw
directory when it is full, in which case it would be flushed/discarded. This implies there is a delay between when the event is logged and when the event is captured.Commit logs on an individual Cassandra node do not reflect all writes to the cluster, they only reflect writes stored on that node. This is why it is necesssary to monitor changes on all nodes in a Cassandra cluster. However, due to replication factor, this also implies it is necessary for downstream consumers of these events to handle deduplication.
Writes to an individual Cassandra node are logged as they arrive. However, these events may arrive out-of-order from which they are issued. Downstream consumers of these events must understand and implement logic similar to Cassandra’s read path to get the correct output.
Schema changes of tables are not recorded in commit logs, only data changes are recorded. Therefore changes in schema are detected on a best-effort basis. To avoid data loss, it is recommended to pause writes to the table during schema change.
Cassandra does not perform read-before-write, as a result commit logs do not record the value of every column in the changed row, it only records the values of columns that have been modified (except for partition key columns, which are always recorded as they are required in Cassandra DML commands).
Due to the nature of CQL, insert DMLs can result in a row insertion or update; update DMLs can result in a row insertion, update, or deletion; delete DMLs can result in a row update or deletion. Since queries are not recorded in commit logs, CDC event type is classified based on the effect on the row in a relational database sense.
TODO: is there a way to determine event type which corresponds to the actual Cassandra DML statement? and if so, is that preferred over the semantic of these events?
In Cassandra 4.0 on every segment fsync, an index file will be updated to reflect latest offset. This will eliminate the processing delay in the CDC feature in Cassandra 3.X. This connector feature is to be added in the future with the Cassandra 4.0 release. |
Managing Commit Log Lifecycle
By default, Cassandra connector will delete commit logs which have been processed. It is not recommended to start the connector while deletion of commit logs is disabled, as this could bloat up disk storage and prevent further writes to the Cassandra cluster. To manage the commit logs in a custom manner (i.e. upload it to a cloud provider), the CommitLogTransfer interface can be implemented.
Topics names
The Cassandra connector writes events for all insert, update, and delete uperations on a single table to a single Kafka topic. The name of the Kafka topics always take the form clusterName.keyspaceName.tableName, where clusterName is the logical name of the connector as specified with the kafka.topic.prefix
configuraiton property, keyspaceName is the name of the keyspace where the operation occurred, and tableName is the name of the table on which the operation occurred.
For example, consider a Cassandra installation with an inventory
keyspace that contains four tables: products
, products_on_hand
, customers
, and orders
. If the connector monitoring this database were given a logical server name of fulfillment
, then the connector would produce events on these four Kafka topics:
fulfillment.inventory.products
fulfillment.inventory.products_on_hand
fulfillment.inventory.customers
fulfillment.inventory.orders
TODO: for topic name, is clusterName.keyspaceName.tableName okay? or should it be connectorName.keyspaceName.tableName or connectorName.clusterName.keyspaceName.tableName?
Schema Evolution
DDLs are not recorded in commit logs. When the schema of a table change, this change is issued from one of the Cassandra node and propagated to other nodes via Gossip Protocol. This implies detection of schema changes are achieved on a best-effort basis. This is done by periodically polling the schema of each cdc-enabled table in the cluster via a Cassandra driver, and then update the cached version of the schema. Because of this implementation, if a new column is added to a table and then writes are issued against that column immediately, it is possible that data from that column will not be reflected in the CDC event. This is why it is recommened to pause for some time (configured with schema.refresh.interval.ms
) after issuing a schema change.
TODO: it may be possible to reactively refresh schema whenever an unexpect column appears in a mutation to improve schema change detection; worth looking into.
When sending a message to a topic, the Kafka Connect schema for the key and the value will be automatically registered in the Confluent Schema Registry under the subject t-key and t-value, respectively, if the compatibility test passes. Although it is possible to replay a history of all table schemas via the Schema Registry, only the latest schema of each table is used to generate CDC events.
TODO: look into whether it is possible to leverage schema history to rebuild schema that exist at the specific position in the commit log, rather than the current schema, when restarting the connector. I don’t think it is possible right now, because writes to Cassandra node are not received in order.
Events
All data change events produced by the Cassandra connector have a key and a value, although the structure of the key and value depends on the table from which the change events originated (see Topic Names).
Change Event’s Key
For a given table, the change event’s key will have a structure that contains a field for each column in the primary key of the table at the time the event was created. Consider an inventory
database with a customers
table defined as:
CREATE TABLE customers (
id bigint,
registration_date timestamp,
first_name text,
last_name text,
email text,
PRIMARY KEY (id, registration_date)
);
Every change event for the customers
table while it has this definition will feature the same key schema, which in JSON representation looks like this:
{
"type": "record",
"name": "cassandra-cluster-1.inventory.customers.Key",
"namespace": "io.debezium.connector.cassandra",
"fields": [
{
"name": "id",
"type": "long"
},
{
"name": "registration_date",
"type": "long",
"logicalType": "timestamp-millis"
}
]
}
For id = 1001 and registration_date = 1562202942545, the key payload in JSON representation would look like this:
{
"id": 1001,
"registration_date": 1562202942545
}
Although the |
Change event’s value
The value of the change event message is a bit more complicated. Every change event value produced by Cassandra 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 Cassandra connector arei
for insert,u
for update, andd
for delete.after
is an optional field that if present contains the state of the row after the event occurred. The structure will be described by thecassandra-cluster-1.inventory.customers.Value
Kafka Connect schema, which represent the cluster, keyspace, and table the event is referring to.source
is a mandatory field that contains a structure describing the source metadata for the event, which in the case of Cassandra contains several fields: the Debezium version, the connector name, the Cassandra cluster name, the name of the commit log file where the event was recorded, the position in that commit log file where the event appeared, whether this event was part of a snapshot, name of the affected keyspace and table, and the maximum timestamp of the partition update in microseconds.ts_ms
is optional and if present contains the time (using the system clock in the JVM running the Cassandra connector) at which the connector processed the event.
Note that there is no |
The following is a JSON representation of a value schema for a create event for our customers
table:
{
"type": "record",
"name": "cassandra-cluster-1.inventory.customers.Value",
"namespace": "io.debezium.connector.cassandra",
"fields": [
{
"name": "op",
"type": "string"
},
{
"name": "ts_ms",
"type": "long",
"logicalType": "timestamp-millis"
},
{
"name": "after",
"type": "record",
"fields": [
{
"name": "id",
"type": [
"null",
{
"name": "id",
"type": "record",
"fields": [
{
"name":"value",
"type": "string"
},
{
"name":"deletion_ts",
"type": ["null", "long"],
"default" : "null"
},
{
"name":"set",
"type": "boolean"
}
]
}
]
},
{
"name": "registration_date",
"type": [
"null",
{
"name": "registration_date",
"type": "record",
"fields": [
{
"name":"value",
"type": "long",
"logical_type": "timestamp-millis"
},
{
"name":"deletion_ts",
"type": ["null", "long"],
"default" : "null"
},
{
"name":"set",
"type": "boolean"
}
]
}
]
},
{
"name": "first_name",
"type": [
"null",
{
"name": "first_name",
"type": "record",
"fields": [
{
"name":"value",
"type": "string"
},
{
"name":"deletion_ts",
"type": ["null", "long"],
"default" : "null"
},
{
"name":"set",
"type": "boolean"
}
]
}
]
},
{
"name": "last_name",
"type": [
"null",
{
"name": "last_name",
"type": "record",
"fields": [
{
"name":"value",
"type": "string"
},
{
"name":"deletion_ts",
"type": ["null", "long"],
"default" : "null"
},
{
"name":"set",
"type": "boolean"
}
]
}
]
},
{
"name": "last_name",
"type": [
"null",
{
"name": "email",
"type": "record",
"fields": [
{
"name":"value",
"type": "string"
},
{
"name":"deletion_ts",
"type": ["null", "long"],
"default" : "null"
},
{
"name":"set",
"type": "boolean"
}
]
}
]
}
]
},
{
"name": "source",
"type": "record",
"fields": [
{
"name": "version",
"type": "string"
},
{
"name": "connector",
"type": "string"
},
{
"name": "cluster",
"type": "string"
},
{
"name": "snapshot",
"type": "boolean"
},
{
"name": "keyspace",
"type": "string"
},
{
"name": "table",
"type": "string"
},
{
"name": "file",
"type": "string"
},
{
"name": "position",
"type": "int"
},
{
"name": "ts_micro",
"type": "long",
"logicalType": "timestamp-micros"
}
]
}
]
}
TODO: verify max timestamp != deletion timestamp in case of deletion DDLs
Given the following insert
DML:
INSERT INTO customers (
id,
registration_date,
first_name,
last_name,
email)
VALUES (
1001,
now(),
"Anne",
"Kretchmar",
"annek@noanswer.org"
);
The value payload in JSON representation would look like this:
{
"op": "c",
"ts_ms": 1562202942832,
"after": {
"id": {
"value": 1001,
"deletion_ts": null,
"set": true
},
"registration_date": {
"value": 1562202942545,
"deletion_ts": null,
"set": true
},
"first_name": {
"value": "Anne",
"deletion_ts": null,
"set": true
},
"last_name": {
"value": "Kretchmar",
"deletion_ts": null,
"set": true
},
"email": {
"value": "annek@noanswer.org",
"deletion_ts": null,
"set": true
}
},
"source": {
"version": "0.9.5.Final",
"connector": "cassandra",
"cluster": "cassandra-cluster-1",
"snapshot": false,
"keyspace": "inventory",
"table": "customers",
"file": "commitlog-6-123456.log",
"pos": 54,
"ts_micro": 1562202942666382
}
}
Given the following update
DML:
UPDATE customers
SET email = "annek_new@noanswer.org"
WHERE id = 1001 AND registration_date = 1562202942545
The value payload in JSON representation would look like this:
{
"op": "u",
"ts_ms": 1562202942912,
"after": {
"id": {
"value": 1001,
"deletion_ts": null,
"set": true
},
"registration_date": {
"value": 1562202942545,
"deletion_ts": null,
"set": true
},
"first_name": null,
"last_name": null,
"email": {
"value": "annek_new@noanswer.org",
"deletion_ts": null,
"set": true
}
},
"source": {
"version": "0.9.5.Final",
"connector": "cassandra",
"cluster": "cassandra-cluster-1",
"snapshot": false,
"keyspace": "inventory",
"table": "customers",
"file": "commitlog-6-123456.log",
"pos": 102,
"ts_micro": 1562202942666490
}
}
When we compare this to the value in the insert event, we see a couple differences:
The
op
field value is nowu
, signifying that this row changed because of an update.The
after
field now has the updated state of the row, and here we can see that the email value is nowannek_new@noanswer.org
. Notice thatfirst_name
andlast_name
are null, this is because these fields did not change during this update. However,id
andregistration_date
are still included, because these are the primary keys of this table.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 commit log.The
ts_ms
shows the timestamp milliseconds which the connector processed this event.
Finally, given the following delete
DML:
DELETE FROM customers
WHERE id = 1001 AND registration_date = 1562202942545;
The value payload in JSON representation would look like this:
{
"op": "d",
"ts_ms": 1562202942912,
"after": {
"id": {
"value": 1001,
"deletion_ts": 1562202972545,
"set": true
},
"registration_date": {
"value": 1562202942545,
"deletion_ts": 1562202972545,
"set": true
},
"first_name": null,
"last_name": null,
"email": null
},
"source": {
"version": "0.9.5.Final",
"connector": "cassandra",
"cluster": "cassandra-cluster-1",
"snapshot": false,
"keyspace": "inventory",
"table": "customers",
"file": "commitlog-6-123456.log",
"pos": 102,
"ts_micro": 1562202942666490
}
}
When we compare this to the value in the insert and update event, we see a couple differences:
The
op
field value is nowd
, signifying that this row changed because of a deletion.The
after
field only contains values forid
andregistration_date
because this is a deletion by primary keys.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 commit log.The
ts_ms
shows the timestamp milliseconds which the connector processed this event.
TODO: given TTL is not currently support, would it be better to remove delete_ts? would it also be okay to derive whether a field is set or not by looking at the each column to see if it is null?
TODO: discuss tombstone events in Cassandra connector
Data Types
As described above, the Cassandra 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 Cassandra data type of the column. This section describes this mapping.
The following table describes how the connector maps each of the Cassandra data types to an Kafka Connect data type.
Cassandra Data Type | Literal Type (Schema Type) | Semantic Type (Schema Name) |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
|
|
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
| n/a |
|
|
|
|
|
|
|
| n/a |
|
| n/a |
|
|
|
|
| n/a |
|
| n/a |
|
|
|
TODO: add logical types
When Things Go Wrong
Configuration And Startup Errors
The Cassandra connector will fail upon startup, report error or exception in the log, and stop running if the configurations are invalid or if the connector cannot successfully connector to Cassandra using the specified connectivity parameters. In this case, the error will have more details about the problem and possibly suggest a work around. The connector can be restarted when the configuration has been corrected.
Cassandra Becomes Unavailable
Once the connector is running, if the Cassandra node becomes unavailable for any reason, the connector will fail and stop. In this case, restart the connector once the server becomes available. If this happened during snapshot, it will rebootstrap the entire table from the beginning of the table.
Cassandra Connector Stops Gracefully
If the Cassandra connector is gracefully shut down, prior to stopping the process it will make sure to flush all events in the ChangeEventQueue to Kafka. The Cassandra connector keeps track of the filename and offset each time a streamed record is send to Kafka. So when the connector is restarted, it will resume from where it left off. It does this by searching for the oldest commit log in the directory, start processing that commitlog, skipping the already-read records, until it finds the most recent record that hasn’t been processed. If the Cassandra connector is stopped during snapshot, it will pick up from that table, but will rebootstrap the entire table.
Cassandra Connector Crashes
If the Cassandra connector crashes unexpected, then the Cassandra connector would likely have terminated without recording the most-recently processed offset. In this case, when the connector is restarted, it will resume from the most recent recorded offset. This means duplicates is likely (which is trivial since we already be get duplicates from RF). Note that since the offset is only updated when a record has been successfully send to Kafka, it is okay to lose the un-emitted data in the ChangeEventQueue during a crash, as these events will be recreated.
Kafka Becomes Unavailable
As the connector generate change event, it will publish those events to Kafka using Kafka producer API. If Kafka broker becomes unavailable (producer encounters TimeoutException), the Cassandra connector will repeatedly attempt to reconnect to the broker once per second until a successful retry.
Cassandra connector is Stopped for a Duration
Depending on the write load of a table, when a Cassandra connector is stopped for a long time, it risks into hitting the cdc_total_space_in_mb capacity. Once this upper limit is reached, Cassandra will stop accepting writes for this table; which means it is important to monitor this space while running the Cassandra connector. In the worst case scenario when this happens, the best thing to do is to (1) turn off Cassandra connector (2) disable cdc for the table so it stops generating additional writes (although writes to other cdc-enabled tables on the same node could still affect the commitlog file generation given the commit logs are not filtered) (3) remove the recorded offset from the offset file (4) once the capacity is increased or the directory used space is under control, restart the connector so it will rebootstrap the table.
Cassandra Table CDC is Enabled, Then Temporarily Disabled, And Then Enabled Again
If a Cassandra table temporarily disables CDC and then re-enables it again after some time, it must be re-bootstrapped. To re-bootstrap an individual table, you can manually remove the recorded offset line corresponding to that table from snapshot_offset.properties file.
Deploying A Connector
The Cassandra connector should be deployed each Cassandra node in a Cassandra cluster. The Cassandra connector Jar file takes in a cdc configuration (.properties) file. See see example configurations for reference.
Example configuration
The following represents an example .properties configuration file for running and testing the Cassandra Connector locally:
connector.name=test_connector
commit.log.relocation.dir=/Users/test_user/debezium-connector-cassandra/test_dir/relocation/
http.port=8000
cassandra.config=/usr/local/etc/cassandra/cassandra.yaml
cassandra.hosts=127.0.0.1
cassandra.port=9042
kafka.producer.bootstrap.servers=127.0.0.1:9092
kafka.producer.retries=3
kafka.producer.retry.backoff.ms=1000
kafka.topic.prefix=test_prefix
key.converter=io.confluent.connect.avro.AvroConverter
key.converter.schema.registry.url: http://localhost:8081
value.converter=io.confluent.connect.avro.AvroConverter
value.converter.schema.registry.url: http://localhost:8081
offset.backing.store.dir=/Users/test_user/debezium-connector-cassandra/test_dir/
snapshot.consistency=ONE
snapshot.mode=ALWAYS
latest.commit.log.only=true
Monitoring
Cassandra connector has built-in support for JMX metrics. The Cassandra driver also publishes a number of metrics about the driver’s activities that can be monitored through JMX. The connector has two types of metrics. Snapshot metrics help you monitor the snapshot activity and are available when the connector is performing a snapshot. Binlog metrics help you monitor the progress and activity while the connector reads the Cassandra commit logs.
Snapshot Metrics
Attribute Name | Type | Description |
|
| The total number of tables that are being included in the snapshot. |
|
| The number of tables that the snapshot has yet to copy. |
|
| Whether the snapshot was started. |
|
| Whether the snapshot was aborted. |
|
| Whether the snapshot completed. |
|
| The total number of seconds that the snapshot has taken so far, even if not complete. |
|
| Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table. |
Commitlog Metrics
Attribute Name | Type | Description |
|
| The name of the commit log filename that the connector has most recently read. |
|
| The most recent position (in bytes) within the commit log that the connector has read. |
|
| The number of mutations that have been processed. |
|
| The number of unrecoverable errors while processing commit logs. |
Connector properties
Property | Default | Description |
| Specifies the criteria for running a snapshot (eg. initial sync) upon startup of the cassandra connector agent. Must be one of ‘INITIAL’, ‘ALWAYS’, or ‘NEVER’. The default snapshot mode is ‘INITIAL’. | |
| Specify the {@link ConsistencyLevel} used for the snapshot query. | |
| The port used by the HTTP server for ping, health check, and build info | |
The absolute path of the YAML config file used by a Cassandra node. | ||
| One or more addresses of Cassandra nodes that driver uses to discover topology, separated by “,” | |
| The port used to connect to Cassandra host(s). | |
The username used when connecting to Cassandra hosts. | ||
The password used when connecting to Cassandra hosts. | ||
| If set to true, Cassandra connector agent will use SSL to connect to Cassandra node. | |
The SSL config file path required for storage node. | ||
The local directory which commit logs get relocated to once processed. | ||
| Determines whether or not the CommitLogPostProcessor should run. If disabled, commit logs would not be deleted post-process. | |
| The fully qualified {@link CommitLogTransfer} class used to transfer commit logs. Built-in transfers are | |
The remote base directory to transfer commit log to. Required to be specified if GCSCommitLogTransfer is used as COMMIT_LOG_TRANSFER_CLASS. For example: “gcs://bucket-name/prefix-name”. | ||
The path of the storage key file used for setting up credential to access remote storage service. If {@link GCSCommitLogTransfer} is used and this config is not provided, then Application Default Credential would be used. | ||
The directory to store offset tracking files. | ||
| The minimum amount of time to wait before committing the offset. The default value of 0 implies the offset will be flushed every time. | |
| The maximum records that are allowed to be processed until it is required to flush offset to disk. This config is effective only if offset_flush_interval_ms != 0. | |
| Positive integer value that specifies the maximum size of the blocking queue into which change events read from the commit log are placed before they are written to Kafka. This queue can provide back pressure to the commit log 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. The capacity of the queue to hold deserialized records before they are converted to Kafka Connect structs and emitted to Kafka. | |
| The maximum number of change events to dequeue each time. | |
| Positive integer value that specifies the number of milliseconds the commit log processor should wait during each iteration for new change events to appear in the queue. Defaults to 1000 milliseconds, or 1 second. | |
| Positive integer value that specifies the number of milliseconds the schema processor should wait before refreshing the cached Cassandra table schemas. | |
| The maximum amount of time to wait on each poll before reattempt. | |
| Positive integer value that specifies the number of milliseconds the snapshot processor should wait before re-scanning tables to look for new cdc-enabled tables. Defaults to 10000 milliseconds, or 10 seconds. | |
| Whether deletion events should have a subsequent tombstone event (true) or not (false). It’s important to note that in Cassandra, two events with the same key may be updating different columns of a given table. So this could potentially result in records being lost during compaction if they have not been consumed by the consumer yet. In other words, do NOT set this to true if you have Kafka compaction turned on. | |
A comma-separated list of fully-qualified names of fields that should be excluded from change event message values. Fully-qualified names for fields are in the form keyspace_name>.<field_name>.<nested_field_name>. | ||
| Whether field names will be sanitized to adhere to Avro naming requirements. See Avro naming for more details. |
The connector also supports pass-through configuration properties that are used when creating the Kafka producer. Specifically, all connector configuration properties that begin with the kafka.producer.
prefix are used (without the prefix) when creating the Kafka producer that writes events to Kafka.
For example, the follwoing connector configuration properties can be used to secure connections to the Kafka broker:
kafka.producer.security.protocol=SSL
kafka.producer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
kafka.producer.ssl.keystore.password=test1234
kafka.producer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
kafka.producer.ssl.truststore.password=test1234
kafka.producer.ssl.key.password=test1234
kafka.consumer.security.protocol=SSL
kafka.consumer.ssl.keystore.location=/var/private/ssl/kafka.server.keystore.jks
kafka.consumer.ssl.keystore.password=test1234
kafka.consumer.ssl.truststore.location=/var/private/ssl/kafka.server.truststore.jks
kafka.consumer.ssl.truststore.password=test1234
kafka.consumer.ssl.key.password=test1234
Be sure to consult the Kafka documentation for all of the configuration properties for Kafka producers.
The connector supports the following Kafka Connect converters for key/value serialization:
io.confluent.connect.avro.AvroConverter
org.apache.kafka.connect.storage.StringConverter
org.apache.kafka.connect.json.JsonConverter
com.blueapron.connect.protobuf.ProtobufConverter