About CDC for Cassandra

DataStax CDC for Apache Cassandra® is open-source software (OSS) that sends Cassandra mutations for tables having Change Data Capture (CDC) enabled to Luna Streaming or Apache Pulsar™, which in turn can write the data to platforms such as Elasticsearch® or Snowflake®.

Key Features

  • Supports Apache Cassandra 3.11+, 4.0+, and Datastax Enterprise Server 6.8.16+

  • Supports Luna Streaming 2.8+ and Apache Pulsar 2.8.1+

  • De-duplicates updates from multiple replicas

  • Propagates Cassandra schema change to the built-in Pulsar schema registry

  • Supports AVRO message format

Architecture

Other than the prerequisite C* and Pulsar clusters, CDC for Cassandra has two components:

  • DataStax Change Agent for Apache Cassandra, which is an event producer deployed as a JVM agent on each Cassandra data node

  • CDC for Cassandra, which is a source connector deployed in your Pulsar cluster

The following diagram describes the general architecture.

cdc for cassandra overview

Since version 3.0, Cassandra has included a change data capture (CDC) feature. 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 discard/flush (default: cdc_raw). Since Cassandra 4.0 (and back ported in C* 3.11+), the C* CDC feature has been improved such that commit logs can be synced to the cdc_raw directory periodically (by default, 10 seconds). This improvement makes the C* CDC feature near real-time.

When CDC is enabled:

  1. The change agent running on each node reads the Cassandra commitlog files in the cdc_raw directory and emits a message for each Cassandra mutation on a CDC-enabled table to a per-table events topic.

  2. The CDC for Cassandra agent processes these events and fetches the mutated Cassandra row.

  3. The Cassandra Source Connector writes the row into the data topic with necessary processing tasks that make sure the most recent state of the Cassandra table is replicated into the data topic correctly (no duplication, right order, etc.)

  4. Once the change agent processes all mutations in the commit log, it deletes the file from the cdc_raw directory.

CDC for Cassandra is tolerant of failures when processing the commit logs in the cdc_raw directory. It maintains a processing offset for each commit log. If the change agent restarts, CDC for Cassandra picks up where it left off using the recorded offset value.

The following table describes what is published to the data topic for each update to a CDC-enabled Cassandra table.

Type

Event Data

insert

Key set to primary key of the row, value set to all column values

update

Key set to primary key of the row, value set to all column values

delete

Key set to primary key of the row, value set to null

The CSC for Pulsar updates the schema registry to dynamically reflect the Cassandra table schema. You can then deploy various sink connectors to replicate data into the backends of your choice. For more, see Pulsar built-in sink connectors.

Sink connectors processing messages from the data topic should interpret an event with a null value as a delete. For example, with the Pulsar Elasticsearch connector, you need to set nullValueAction to DELETE.

The change agent runs on all Cassandra data nodes. This means that the agent processes the original write plus its replicas. To minimize the number of duplicate events that end up in the data topic, the CDC for Cassandra maintains an in-memory de-duplication cache. For each update to the table, an MD5 digest is calculated to de-duplicate the updates from the replicas.

Change Agent deployment matrix

Cassandra version

Apache Pulsar/Luna Streaming

Cassandra v3.x

agent-c3

Cassandra v4.x

agent-c4

DSE 6.8.16+

agent-dse4

Supported streaming platforms

  • Luna Streaming 2.8 and later (current Luna Streaming version is 2.10)

  • Apache Pulsar 2.8.1 and later (current Pulsar version is 2.10)

Connector deployment matrix

Apache Pulsar/Luna Streaming

connector

Supported databases

  • Apache Cassandra® 3.11.x and 4.x databases

  • Datastax Server Enterprise 6.8.16+

Supported Cassandra data structures

The following CQL data types are encoded as AVRO logical types:

  • ascii (string)

  • bigint (long)

  • blob(bytes)

  • boolean (boolean)

  • Collection types:

    • list (array)

    • set (array)

    • map (map)

  • date (date)

  • decimal (cql_decimal)

  • double (double)

  • duration (cql_duration)

  • float (float)

  • inet (string)

  • int (int)

  • smallint (int)

  • text (string)

  • time (time-micros)

  • timestamp (timestamp-millis)

  • timeuuid (uuid)

  • tinyint (int)

  • User Defined Types (record)

  • uuid (uuid)

  • varint (cql_varint)

If using the key-value-json output format, the supported C* types are the same as AVRO. The output is an exact schema with logical types, but with a JSON schema type.

Cassandra static columns are supported:

  • On row-level updates, static columns are included in the message value.

  • On partition-level updates, the clustering keys are null in the message key, and the message value only has static columns on insert/update operations.

For data types that are not supported, columns using those data types are omitted from the events sent to the data topic. If a row update contains both supported and unsupported data types, the event will include only columns with supported data types.

Limitations

CDC for Cassandra has the following limitations:

  • Does not manage table truncates. The TRUNCATE [TABLE] command should not be used.

  • Does not sync data available before starting the CDC agent.

  • Does not replay logged batches.

  • Does not manage time-to-live.

  • Does not support range deletes.

  • CQL column names must not match a Pulsar primitive type name (ex: INT32).

Deployment

Schema updates on topics

Schema registry updates on a Pulsar topic are controlled by the is-allow-auto-update-schema option.

  • true allows the broker to register a new schema for a topic and connect the producer if the schema is not registered.

  • false rejects the producer’s connection to the broker if the schema is not registered.

To ensure the CDC for Cassandra can automatically update the schema on the Pulsar topic, set the option to true. For more, see Schema Auto-Update.

Multiple Cassandra datacenters

In a multi-datacenter Cassandra configuration, enable CDC and install the change agent in only one datacenter. To ensure the data sent to all datacenters are delivered to the data topic, make sure to configure replication to the datacenter that has CDC enabled on the table.

For example, given a Cassandra cluster with three datacenters (DC1, DC2, and DC3), you would enable CDC and install the change agent in only DC1. To ensure all updates in DC2 and DC3 are propagated to the data topic, configure the table’s keyspace to replicate data from DC2 and DC3 to DC1. For example, replication = {'class': 'NetworkTopologyStrategy', 'dc1': 3, 'dc2': 3, 'dc3': 3}). The data replicated to DC1 will be processed by the change agent and eventually end up in the data topic.

What’s next?

  • For more on using CDC with Apache Pulsar, including schema management and consumption patterns, see our Streaming learning page.

  • If you’ve got more questions about DataStax CDC for Apache Cassandra®, see CDC for Cassandra FAQs.

Was this helpful?

Give Feedback

How can we improve the documentation?

© 2024 DataStax | Privacy policy | Terms of use

Apache, Apache Cassandra, Cassandra, Apache Tomcat, Tomcat, Apache Lucene, Apache Solr, Apache Hadoop, Hadoop, Apache Pulsar, Pulsar, Apache Spark, Spark, Apache TinkerPop, TinkerPop, Apache Kafka and Kafka are either registered trademarks or trademarks of the Apache Software Foundation or its subsidiaries in Canada, the United States and/or other countries. Kubernetes is the registered trademark of the Linux Foundation.

General Inquiries: +1 (650) 389-6000, info@datastax.com