ScyllaDB Documentation Logo Documentation
  • Server
    • ScyllaDB Open Source
    • ScyllaDB Enterprise
    • ScyllaDB Alternator
  • Cloud
  • Tools
    • ScyllaDB Manager
    • ScyllaDB Monitoring Stack
    • ScyllaDB Operator
  • Drivers
    • CQL Drivers
    • DynamoDB Drivers
Download
Menu
ScyllaDB Docs ScyllaDB Open Source Scylla for Developers ScyllaDB Features Scylla Open Source Features Change Data Capture (CDC) CDC Stream Generations

CDC Stream Generations¶

Stream IDs used for CDC log entries change over time. A single base partition key might be mapped to one stream in the log today, but to a different stream tomorrow. If you build a query which follows changes made to your favorite partition by using a WHERE clause to specify the proper stream ID, you might need to update the query due to a CDC generation change. The good news is:

  • stream IDs will only change if you join a new node to the cluster,

  • it is easy to learn what the used stream IDs are (Querying CDC Streams).

Note

Stream IDs are chosen to maintain the following invariant:

  • given a base write with partition key pk, the corresponding log table entries will have partition key s_id such that the token of pk is in the same vnode as the token of s_id.

Since adding a node to the cluster splits vnodes into smaller vnodes, we need to (in general) change the used stream IDs.

These sets of stream IDs are called CDC stream generations (also referred to simply as CDC generations).

A CDC generation consists of:

  • a timestamp, describing the point in time from which this generation starts operating,

  • a set of stream IDs,

  • a mapping from the set of tokens (in the entire token ring) to the set of stream IDs in this generation.

This is the mapping used to decide on which stream IDs to use when making writes, as explained in the CDC Streams document. It is a global property of the cluster: it doesn’t depend on the table you’re making writes to.

Caution

The tables mentioned in the following sections: system_distributed.cdc_generation_timestamps and system_distributed.cdc_streams_descriptions_v2 have been introduced in Scylla 4.4. It is highly recommended to upgrade to 4.4 for efficient CDC usage. The last section explains how to run the below examples in Scylla 4.3.

When CDC generations change¶

When you start a fresh cluster, the first generation is created. It has a timestamp chosen using the local clock of the node; it is stored in the time column of the system_distributed.cdc_generation_timestamps table. The stream IDs used in this generation are stored in the streams column of the system_distributed.cdc_streams_descriptions_v2 table. Whenever you bootstrap a new node, you will see a new row appear in cdc_generation_timestamps containing the new generation’s timestamp and a new partition in cdc_streams_descriptions_v2 containing the stream IDs of that new generation.

The cdc_generation_timestamps table is a single-partition table; all timestamps are stored in the key = 'timestamps' partition.

Example: The Next Generation¶

  1. Start a new cluster. Query the cdc_generation_timestamps table to see the available generations:

    SELECT time FROM system_distributed.cdc_generation_timestamps WHERE key = 'timestamps';
    

    returns:

     time
    ---------------------------------
     2020-03-25 12:44:43.006000+0000
    
    (1 rows)
    

    this is the timestamp of the first generation.

  2. Create a table and insert a row:

    CREATE TABLE ks.t (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH cdc = {'enabled': true};
    INSERT INTO ks.t (pk, ck, v) values (0,0,0);
    
  3. Bootstrap another node. After it finishes joining the cluster, query cdc_generation_timestamps again:

    SELECT time FROM system_distributed.cdc_generation_timestamps WHERE key = 'timestamps';
    

    returns:

     time
    ---------------------------------
     2020-03-25 16:05:29.484000+0000
     2020-03-25 12:44:43.006000+0000
    
    (2 rows)
    

    Note that time is the clustering key column of this table. It is sorted in descending order.

  4. Wait until the new generation starts operating. You can do that by using the CQL now() function to periodically check the current time of the node you’re connected to:

    SELECT totimestamp(now()) FROM system.local;
    

    returns (after waiting):

     system.totimestamp(system.now())
    ----------------------------------
      2020-03-25 16:05:31.939000+0000
    
    (1 rows)
    
  5. Insert a row to your table again:

    INSERT INTO ks.t (pk, ck, v) values (0,0,0);
    
  6. Query the log table:

    SELECT "cdc$stream_id", pk FROM ks.t_scylla_cdc_log;
    

    returns:

     cdc$stream_id                      | pk
    ------------------------------------+----
     0x0521d5ce4a4a8ca552f83d88a1ae55d2 |  0
     0x166eddaa68db9a95af83968998626f7c |  0
    
    (2 rows)
    

    There are two entries with the same base partition key, but in different streams. One of them corresponds to the write made before the generation change, the other — to the write made after the change.

After the operating CDC generation changes, all writes with timestamps greater than or equal to the new generation’s timestamp will use the new stream IDs. If you try to perform a write with a timestamp that is smaller than the new generation’s timestamp, the write may be rejected, depending on the node you’re connected to:

  • if the clock of the node you’re connected to reports earlier time than the generation’s timestamp, it will allow the write to be performed.

  • Otherwise, the write will be rejected.

Therefore, if you’ve configured the driver to generate timestamps for you, make sure that the clock of the machine your driver is running on is not too desynchronized with the clock of the node you’re connecting to. That way you can minimize the chance of writes being rejected while a new node is being bootstrapped.

Example: rejecting writes to an old generation¶

This is a continuation of the previous example; a second node was bootstrapped recently, thus a new generation superseded the previous one.

  1. Get the timestamp of the latest generation as an integer:

    SELECT tounixtimestamp(time) FROM system_distributed.cdc_generation_timestamps WHERE key = 'timestamps';
    

    result:

     system.tounixtimestamp(time)
    ------------------------------
                    1585152329484
                    1585140283006
    
    (2 rows)
    

    Generation timestamps have millisecond resolution. Here, the latest generation’s timestamp is equal to 1585152329484 milliseconds.

  2. Try to perform a write with a slightly smaller timestamp (remember that the USING TIMESTAMP clause expects a timestamp in microseconds):

    INSERT INTO ks.t (pk, ck, v) VALUES (0, 0, 0) USING TIMESTAMP 1585152329483000;
    

    result:

    InvalidRequest: Error from server: code=2200 [Invalid query] message="cdc: attempted to get a stream from an earlier generation than the currently used one. With CDC you cannot send writes with timestamps too far into the past, because that would break consistency properties (write timestamp: 2020/03/25 16:05:29, current generation started at: 2020/03/25 16:05:29)"
    

    The write was rejected.

  3. Perform a write with a timestamp equal to the generation’s timestamp:

    INSERT INTO ks.t (pk, ck, v) VALUES (0, 0, 0) USING TIMESTAMP 1585152329484000;
    

    The write succeeds.

The first generation’s timestamp¶

The first generation’s timestamp is chosen by the first starting node by taking the current time (on the node’s clock) shifted forward by a small duration (around a minute). Therefore you won’t be able to perform writes to CDC-enabled tables immediately after starting the first node: there is no CDC generation operating yet.

Example: “could not find any CDC stream”¶

Suppose a node was started at 17:59:35 UTC+1 time, as reported by the node’s logs:

INFO  2020-02-06 17:59:35,087 [shard 0] init - Scylla version 666.development-0.20200206.9eae0b57a with build-id 052adc1eb0601af2 starting ...

You immediately connected to the node using cqlsh and queried the cdc_generation_timestamps table:

SELECT time FROM system_distributed.cdc_generation_timestamps WHERE key = 'timestamps';

The result was:

 time
---------------------------------
 2020-02-06 17:00:43.100000+0000

(1 rows)

This generation’s timestamp is 17:00:43 UTC time (timestamp columns in Scylla always show the timestamp as a UTC time-date), which is a little more than a minute later compared to the node’s startup time (which was 16:59:35 UTC time).

If you then immediately create a CDC-enabled table and attempt to make an insert:

CREATE KEYSPACE ks WITH replication = {'class':'SimpleStrategy', 'replication_factor': 1};
CREATE TABLE ks.t (pk int, ck int, v int, PRIMARY KEY (pk, ck)) WITH cdc = {'enabled': true};
INSERT INTO ks.t (pk, ck, v) values (0, 0, 0);

the result will be an error message:

ServerError: cdc::metadata::get_stream: could not find any CDC stream (current time: 2020/02/06 16:59:58). Are we in the middle of a cluster upgrade?

If you see a message like that, it doesn’t necessarily mean something is wrong, as it may simply mean that the first generation hasn’t started operating yet. If you wait for about a minute, you should be able to write to a CDC-enabled table.

You may also see this message if you were running a cluster with an old version of Scylla (which didn’t support CDC) and started a rolling upgrade. Make sure to upgrade all nodes before you start doing CDC writes: one of the nodes will be responsible for creating the first CDC generation and informing other nodes about it.

Differences in Scylla 4.3¶

In Scylla 4.3 the tables cdc_generation_timestamps and cdc_streams_descriptions_v2 don’t exist. Instead there is the cdc_streams_descriptions table. To retrieve all generation timestamps, instead of querying the time column of cdc_generation_timestamps using a single-partition query (i.e. using WHERE key = 'timestamps'), you would query the time column of cdc_streams_descriptions with a full range scan (without specifying a single partition):

SELECT time FROM system_distributed.cdc_streams_descriptions;

Unfortunately, the time column is the partition key column of this table. Therefore the values are not sorted, unlike the values of the time column of the cdc_generation_timestamps table (in which time is the clustering key). You will have to sort them yourselves in order to learn the timestamp of the last generation. Furthermore, querying the table with a full range scan like above requires the coordinator to contact the entire cluster, potentially increasing resource usage and latency. Thus we recommend upgrading to Scylla 4.4 and use the new description tables instead.

PREVIOUS
CDC Streams
NEXT
Querying CDC Streams
ScyllaDB Open Source
  • 5.1
    • master
    • 5.2
    • 5.1
  • Getting Started
    • Install Scylla
      • ScyllaDB Web Installer for Linux
      • Scylla Unified Installer (relocatable executable)
      • Air-gapped Server Installation
      • What is in each RPM
      • Scylla Housekeeping and how to disable it
      • Scylla Developer Mode
      • Scylla Configuration Reference
    • Configure Scylla
    • ScyllaDB Requirements
      • System Requirements
      • OS Support by Platform and Version
      • Scylla in a Shared Environment
    • Migrate to ScyllaDB
      • Migration Process from Cassandra to Scylla
      • Scylla and Apache Cassandra Compatibility
      • Migration Tools Overview
    • Integration Solutions
      • Integrate Scylla with Spark
      • Integrate Scylla with KairosDB
      • Integrate Scylla with Presto
      • Integrate Scylla with Elasticsearch
      • Integrate Scylla with Kubernetes
      • Integrate Scylla with the JanusGraph Graph Data System
      • Integrate Scylla with DataDog
      • Integrate Scylla with Kafka
      • Integrate Scylla with IOTA Chronicle
      • Integrate Scylla with Spring
      • Shard-Aware Kafka Connector for Scylla
      • Install Scylla with Ansible
      • Integrate Scylla with Databricks
    • Tutorials
  • Scylla for Administrators
    • Administration Guide
    • Procedures
      • Cluster Management
      • Backup & Restore
      • Change Configuration
      • Maintenance
      • Best Practices
      • Benchmarking Scylla
      • Migrate from Cassandra to Scylla
      • Disable Housekeeping
    • Security
      • Scylla Security Checklist
      • Enable Authentication
      • Enable and Disable Authentication Without Downtime
      • Generate a cqlshrc File
      • Reset Authenticator Password
      • Enable Authorization
      • Grant Authorization CQL Reference
      • Role Based Access Control (RBAC)
      • Scylla Auditing Guide
      • Encryption: Data in Transit Client to Node
      • Encryption: Data in Transit Node to Node
      • Generating a self-signed Certificate Chain Using openssl
      • Encryption at Rest
      • LDAP Authentication
      • LDAP Authorization (Role Management)
    • Admin Tools
      • Nodetool Reference
      • CQLSh
      • REST
      • Tracing
      • Scylla SStable
      • Scylla Types
      • SSTableLoader
      • cassandra-stress
      • SSTabledump
      • SSTable2json
      • SSTable Index
      • Scylla Logs
      • Seastar Perftune
      • Virtual Tables
    • ScyllaDB Monitoring Stack
    • ScyllaDB Operator
    • ScyllaDB Manager
    • Upgrade Procedures
      • Scylla Enterprise
      • Scylla Open Source
      • Scylla Open Source to Scylla Enterprise
      • Scylla AMI
    • System Configuration
      • System Configuration Guide
      • scylla.yaml
      • Scylla Snitches
    • Benchmarking Scylla
  • Scylla for Developers
    • Learn To Use Scylla
      • Scylla University
      • Course catalog
      • Scylla Essentials
      • Basic Data Modeling
      • Advanced Data Modeling
      • MMS - Learn by Example
      • Care-Pet an IoT Use Case and Example
    • Scylla Alternator
    • Scylla Features
      • Scylla Open Source Features
      • Scylla Enterprise Features
    • Scylla Drivers
      • Scylla CQL Drivers
      • Scylla DynamoDB Drivers
  • CQL Reference
    • CQLSh: the CQL shell
    • Appendices
    • Compaction
    • Consistency Levels
    • Consistency Level Calculator
    • Data Definition
    • Data Manipulation
    • Data Types
    • Definitions
    • Global Secondary Indexes
    • Additional Information
    • Expiring Data with Time to Live (TTL)
    • Additional Information
    • Functions
    • JSON Support
    • Materialized Views
    • Non-Reserved CQL Keywords
    • Reserved CQL Keywords
    • ScyllaDB CQL Extensions
  • Scylla Architecture
    • Scylla Ring Architecture
    • Scylla Fault Tolerance
    • Consistency Level Console Demo
    • Scylla Anti-Entropy
      • Scylla Hinted Handoff
      • Scylla Read Repair
      • Scylla Repair
    • SSTable
      • Scylla SSTable - 2.x
      • ScyllaDB SSTable - 3.x
    • Compaction Strategies
    • Raft Consensus Algorithm in ScyllaDB
  • Troubleshooting Scylla
    • Errors and Support
      • Report a Scylla problem
      • Error Messages
      • Change Log Level
    • Scylla Startup
      • Ownership Problems
      • Scylla will not Start
      • Scylla Python Script broken
    • Cluster and Node
      • Failed Decommission Problem
      • Cluster Timeouts
      • Node Joined With No Data
      • SocketTimeoutException
      • NullPointerException
    • Data Modeling
      • Scylla Large Partitions Table
      • Scylla Large Rows and Cells Table
      • Large Partitions Hunting
    • Data Storage and SSTables
      • Space Utilization Increasing
      • Disk Space is not Reclaimed
      • SSTable Corruption Problem
      • Pointless Compactions
      • Limiting Compaction
    • CQL
      • Time Range Query Fails
      • COPY FROM Fails
      • CQL Connection Table
      • Reverse queries fail
    • Scylla Monitor and Manager
      • Manager and Monitoring integration
      • Manager lists healthy nodes as down
  • Knowledge Base
    • Upgrading from experimental CDC
    • Compaction
    • Counting all rows in a table is slow
    • CQL Query Does Not Display Entire Result Set
    • When CQLSh query returns partial results with followed by “More”
    • Run Scylla and supporting services as a custom user:group
    • Decoding Stack Traces
    • Snapshots and Disk Utilization
    • DPDK mode
    • Debug your database with Flame Graphs
    • How to Change gc_grace_seconds for a Table
    • Gossip in Scylla
    • Increase Permission Cache to Avoid Non-paged Queries
    • How does Scylla LWT Differ from Apache Cassandra ?
    • Map CPUs to Scylla Shards
    • Scylla Memory Usage
    • NTP Configuration for Scylla
    • Updating the Mode in perftune.yaml After a ScyllaDB Upgrade
    • POSIX networking for Scylla
    • Scylla consistency quiz for administrators
    • Recreate RAID devices
    • How to Safely Increase the Replication Factor
    • Scylla and Spark integration
    • Increase Scylla resource limits over systemd
    • Scylla Seed Nodes
    • How to Set up a Swap Space
    • Scylla Snapshots
    • Scylla payload sent duplicated static columns
    • Stopping a local repair
    • System Limits
    • How to flush old tombstones from a table
    • Time to Live (TTL) and Compaction
    • Scylla Nodes are Unresponsive
    • Update a Primary Key
    • Using the perf utility with Scylla
    • Configure Scylla Networking with Multiple NIC/IP Combinations
  • ScyllaDB University
  • Scylla FAQ
  • Contribute to ScyllaDB
  • Glossary
  • Alternator: DynamoDB API in Scylla
    • Getting Started With ScyllaDB Alternator
    • Scylla Alternator for DynamoDB users
  • Create an issue
  • Edit this page

On this page

  • CDC Stream Generations
    • When CDC generations change
      • Example: The Next Generation
      • Example: rejecting writes to an old generation
    • The first generation’s timestamp
      • Example: “could not find any CDC stream”
    • Differences in Scylla 4.3
Logo
Docs Contact Us About Us
Mail List Icon Slack Icon Forum Icon
© 2023, ScyllaDB. All rights reserved.
Last updated on 27 Mar 2023.
Powered by Sphinx 4.3.2 & ScyllaDB Theme 1.3.5