ScyllaDB University LIVE, FREE Virtual Training Event | March 21
Register for Free
ScyllaDB Documentation Logo Documentation
  • Server
  • Cloud
  • Tools
    • ScyllaDB Manager
    • ScyllaDB Monitoring Stack
    • ScyllaDB Operator
  • Drivers
    • CQL Drivers
    • DynamoDB Drivers
  • Resources
    • ScyllaDB University
    • Community Forum
    • Tutorials
Download
ScyllaDB Docs ScyllaDB Enterprise ScyllaDB Architecture Raft Consensus Algorithm in ScyllaDB

Caution

You're viewing documentation for a previous version. Switch to the latest stable version.

Raft Consensus Algorithm in ScyllaDB¶

Introduction¶

ScyllaDB was originally designed, following Apache Cassandra, to use gossip for topology and schema updates and the Paxos consensus algorithm for strong data consistency (LWT). To achieve stronger consistency without performance penalty, ScyllaDB has turned to Raft - a consensus algorithm designed as an alternative to both gossip and Paxos.

Raft is a consensus algorithm that implements a distributed, consistent, replicated log across members (nodes). Raft implements consensus by first electing a distinguished leader, then giving the leader complete responsibility for managing the replicated log. The leader accepts log entries from clients, replicates them on other servers, and tells servers when it is safe to apply log entries to their state machines.

Raft uses a heartbeat mechanism to trigger a leader election. All servers start as followers and remain in the follower state as long as they receive valid RPCs (heartbeat) from a leader or candidate. A leader sends periodic heartbeats to all followers to maintain his authority (leadership). Suppose a follower receives no communication over a period called the election timeout. In that case, it assumes no viable leader and begins an election to choose a new leader.

Leader selection is described in detail in the Raft paper.

ScyllaDB uses Raft to:

  • Manage schema updates in every node. Any schema update, like ALTER, CREATE or DROP TABLE, is first committed as an entry in the replicated Raft log, and, once stored on most replicas, applied to all nodes in the same order, even in the face of a node or network failures.

  • Manage cluster topology. All topology operations are consistently sequenced, making topology updates fast and safe.

Quorum Requirement¶

Raft requires at least a quorum of nodes in a cluster to be available. If multiple nodes fail and the quorum is lost, the cluster is unavailable for schema updates or topology changes. See Handling Failures for information on how to handle failures.

Note that when you have a two-DC cluster with the same number of nodes in each DC, the cluster will lose the quorum if one of the DCs is down. We recommend configuring three DCs per cluster to ensure that the cluster remains available and operational when one DC is down. In the case of two-DC cluster with the same number of nodes in each DC, it is sufficient for the third DC to contain only one node. That node can be configured with join_ring=false and run on a weaker machine. See Configuration Parameters for details about the join_ring option.

Safe Schema Changes with Raft¶

In ScyllaDB, schema is based on Data Definition Language (DDL). In earlier ScyllaDB versions, schema changes were tracked via the gossip protocol, which might lead to schema conflicts if the updates are happening concurrently.

Implementing Raft eliminates schema conflicts and allows full automation of DDL changes under any conditions, as long as a quorum of nodes in the cluster is available. The following examples illustrate how Raft provides the solution to problems with schema changes.

  • A network partition may lead to a split-brain case, where each subset of nodes has a different version of the schema.

    With Raft, after a network split, the majority of the cluster can continue performing schema changes, while the minority needs to wait until it can rejoin the majority. Data manipulation statements on the minority can continue unaffected, provided the quorum requirement is satisfied.

  • Two or more conflicting schema updates are happening at the same time. For example, two different columns with the same definition are simultaneously added to the cluster. There is no effective way to resolve the conflict - the cluster will employ the schema with the most recent timestamp, but changes related to the shadowed table will be lost.

    With Raft, concurrent schema changes are safe.

In summary, Raft makes schema changes safe, but it requires that a quorum of nodes in the cluster is available.

Verifying that the Raft upgrade procedure finished successfully¶

You may need to perform the following procedure as part of the manual recovery procedure.

The Raft upgrade procedure requires full cluster availability to correctly setup the Raft algorithm; after the setup finishes, Raft can proceed with only a majority of nodes, but this initial setup is an exception. An unlucky event, such as a hardware failure, may cause one of your nodes to fail. If this happens before the Raft upgrade procedure finishes, the procedure will get stuck and your intervention will be required.

To verify that the procedure finishes, look at the log of every ScyllaDB node (using journalctl _COMM=scylla). Search for the following patterns:

  • Starting internal upgrade-to-raft procedure denotes the start of the procedure,

  • Raft upgrade finished denotes the end.

The following is an example of a log from a node which went through the procedure correctly. Some parts were truncated for brevity:

features - Feature SUPPORTS_RAFT_CLUSTER_MANAGEMENT is enabled
raft_group0 - finish_setup_after_join: SUPPORTS_RAFT feature enabled. Starting internal upgrade-to-raft procedure.
raft_group0_upgrade - starting in `use_pre_raft_procedures` state.
raft_group0_upgrade - Waiting until everyone is ready to start upgrade...
raft_group0_upgrade - Joining group 0...
raft_group0 - server 624fa080-8c0e-4e3d-acf6-10af473639ca joined group 0 with group id 8f8a1870-5c4e-11ed-bb13-fe59693a23c9
raft_group0_upgrade - Waiting until every peer has joined Raft group 0...
raft_group0_upgrade - Every peer is a member of Raft group 0.
raft_group0_upgrade - Waiting for schema to synchronize across all nodes in group 0...
raft_group0_upgrade - synchronize_schema: my version: a37a3b1e-5251-3632-b6b4-a9468a279834
raft_group0_upgrade - synchronize_schema: schema mismatches: {}. 3 nodes had a matching version.
raft_group0_upgrade - synchronize_schema: finished.
raft_group0_upgrade - Entering synchronize state.
raft_group0_upgrade - Schema changes are disabled in synchronize state. If a failure makes us unable to proceed, manual recovery will be required.
raft_group0_upgrade - Waiting for all peers to enter synchronize state...
raft_group0_upgrade - All peers in synchronize state. Waiting for schema to synchronize...
raft_group0_upgrade - synchronize_schema: collecting schema versions from group 0 members...
raft_group0_upgrade - synchronize_schema: collected remote schema versions.
raft_group0_upgrade - synchronize_schema: my version: a37a3b1e-5251-3632-b6b4-a9468a279834
raft_group0_upgrade - synchronize_schema: schema mismatches: {}. 3 nodes had a matching version.
raft_group0_upgrade - synchronize_schema: finished.
raft_group0_upgrade - Schema synchronized.
raft_group0_upgrade - Raft upgrade finished.

In a functioning cluster with good network connectivity the procedure should take no more than a few seconds. Network issues may cause the procedure to take longer, but if all nodes are alive and the network is eventually functional (each pair of nodes is eventually connected), the procedure will eventually finish.

Note the following message, which appears in the log presented above:

Schema changes are disabled in synchronize state. If a failure makes us unable to proceed, manual recovery will be required.

During the procedure, there is a brief window while schema changes are disabled. This is when the schema change mechanism switches from the older unsafe algorithm to the safe Raft-based algorithm. If everything runs smoothly, this window will be unnoticeable; the procedure is designed to minimize that window’s length. However, if the procedure gets stuck e.g. due to network connectivity problem, ScyllaDB will return the following error when trying to perform a schema change during this window:

Cannot perform schema or topology changes during this time; the cluster is currently upgrading to use Raft for schema operations.
If this error keeps happening, check the logs of your nodes to learn the state of upgrade. The upgrade procedure may get stuck
if there was a node failure.

In the next example, one of the nodes had a power outage before the procedure could finish. The following shows a part of another node’s logs:

raft_group0_upgrade - Entering synchronize state.
raft_group0_upgrade - Schema changes are disabled in synchronize state. If a failure makes us unable to proceed, manual recovery will be required.
raft_group0_upgrade - Waiting for all peers to enter synchronize state...
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: node 127.90.69.3 not in synchronize state yet...
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: node 127.90.69.1 not in synchronize state yet...
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: retrying in a while...
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: node 127.90.69.1 not in synchronize state yet...
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: retrying in a while...
...
raft_group0_upgrade - Raft upgrade procedure taking longer than expected. Please check if all nodes are live and the network is healthy. If the upgrade procedure does not progress even though the cluster is healthy, try performing a rolling restart of the cluster. If that doesn 't help or some nodes are dead and irrecoverable, manual recovery may be required. Consult the relevant documentation.
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: node 127.90.69.1 not in synchronize state yet...
raft_group0_upgrade - wait_for_peers_to_enter_synchronize_state: retrying in a while...

Note the following message:

raft_group0_upgrade - Raft upgrade procedure taking longer than expected. Please check if all nodes are live and the network is healthy. If the upgrade procedure does not progress even though the cluster is healthy, try performing a rolling restart of the cluster. If that doesn 't help or some nodes are dead and irrecoverable, manual recovery may be required. Consult the relevant documentation.

If the Raft upgrade procedure is stuck, this message will appear periodically in each node’s logs.

The message suggests the initial course of action:

  • Check if all nodes are alive.

  • If a node is down but can be restarted, restart it.

  • If all nodes are alive, ensure that the network is healthy: that every node is reachable from every other node.

  • If all nodes are alive and the network is healthy, perform a rolling restart of the cluster.

One of the reasons why the procedure may get stuck is a pre-existing problem in schema definitions which causes schema to be unable to synchronize in the cluster. The procedure cannot proceed unless it ensures that schema is synchronized. If all nodes are alive and the network is healthy, you performed a rolling restart, but the issue still persists, contact ScyllaDB support for assistance.

If some nodes are dead and irrecoverable, you’ll need to perform a manual recovery procedure. Consult the section about Raft recovery.

Consistent Topology with Raft¶

ScyllaDB can use Raft to manage cluster topology. With Raft-managed topology enabled, all topology operations are internally sequenced in a consistent way. A centralized coordination process ensures that topology metadata is synchronized across the nodes on each step of a topology change procedure. This makes topology updates fast and safe, as the cluster administrator can trigger many topology operations concurrently, and the coordination process will safely drive all of them to completion. For example, multiple nodes can be bootstrapped concurrently, which couldn’t be done with the old gossip-based topology.

The feature is automatically enabled in new clusters.

Verifying that Raft is Enabled¶

Schema on Raft

You can verify that Raft is enabled on your cluster by performing the following query on each node:

cqlsh> SELECT * FROM system.scylla_local WHERE key = 'group0_upgrade_state';

The query should return:

 key                  | value
----------------------+--------------------------
 group0_upgrade_state | use_post_raft_procedures

(1 rows)

on every node.

If the query returns 0 rows, or value is synchronize or use_pre_raft_procedures, it means that the cluster is in the middle of the Raft upgrade procedure; consult the relevant section.

If value is recovery, it means that the cluster is in the middle of the manual recovery procedure. The procedure must be finished. Consult the section about Raft recovery.

If value is anything else, it might mean data corruption or a mistake when performing the manual recovery procedure. The value will be treated as if it was equal to recovery when the node is restarted.

Consistent topology changes

You can verify that consistent topology management is enabled on your cluster in two ways:

  1. By querying the system.topology table:

    cqlsh> SELECT upgrade_state FROM system.topology;
    

    The query should return done after upgrade is complete:

    upgrade_state
    ---------------
            done
    
    (1 rows)
    

    An empty result or a value of not_upgraded means that upgrade has not started yet. Any other value means that upgrade is in progress.

  2. By sending a GET HTTP request to the /`storage_service/raft_topology/upgrade` endpoint. For example, you can do it with curl like this:

    curl -X GET "http://127.0.0.1:10000/storage_service/raft_topology/upgrade"
    

    It returns a JSON string, with the same meaning and value as the upgrade_state column in system.topology (see the previous point).

Handling Failures¶

See Handling Node Failures.

Learn More About Raft¶

  • The Raft Consensus Algorithm

  • Achieving NoSQL Database Consistency with Raft in ScyllaDB - A tech talk by Konstantin Osipov

  • Making Schema Changes Safe with Raft - A ScyllaDB Summit talk by Konstantin Osipov (register for access)

  • The Future of Consensus in ScyllaDB 5.0 and Beyond - A ScyllaDB Summit talk by Tomasz Grabiec (register for access)

Was this page helpful?

PREVIOUS
Choose a Compaction Strategy
NEXT
Zero-token Nodes
  • Create an issue

On this page

  • Raft Consensus Algorithm in ScyllaDB
    • Introduction
    • Quorum Requirement
    • Safe Schema Changes with Raft
      • Verifying that the Raft upgrade procedure finished successfully
    • Consistent Topology with Raft
    • Verifying that Raft is Enabled
    • Handling Failures
    • Learn More About Raft
ScyllaDB Enterprise
  • enterprise
    • 2024.2
    • 2024.1
    • 2023.1
    • 2022.2
  • Getting Started
    • Install ScyllaDB Enterprise
      • ScyllaDB Web Installer for Linux
      • Install ScyllaDB Without root Privileges
      • Install scylla-jmx Package
      • Air-gapped Server Installation
      • ScyllaDB Housekeeping and how to disable it
      • ScyllaDB Developer Mode
      • Launch ScyllaDB on AWS
      • Launch ScyllaDB on GCP
      • Launch ScyllaDB on Azure
    • Configure ScyllaDB
    • ScyllaDB Configuration Reference
    • ScyllaDB Requirements
      • System Requirements
      • OS Support
      • Cloud Instance Recommendations
      • ScyllaDB in a Shared Environment
    • Migrate to ScyllaDB
      • Migration Process from Cassandra to ScyllaDB
      • ScyllaDB and Apache Cassandra Compatibility
      • Migration Tools Overview
    • Integration Solutions
      • Integrate ScyllaDB with Spark
      • Integrate ScyllaDB with KairosDB
      • Integrate ScyllaDB with Presto
      • Integrate ScyllaDB with Elasticsearch
      • Integrate ScyllaDB with Kubernetes
      • Integrate ScyllaDB with the JanusGraph Graph Data System
      • Integrate ScyllaDB with DataDog
      • Integrate ScyllaDB with Kafka
      • Integrate ScyllaDB with IOTA Chronicle
      • Integrate ScyllaDB with Spring
      • Shard-Aware Kafka Connector for ScyllaDB
      • Install ScyllaDB with Ansible
      • Integrate ScyllaDB with Databricks
      • Integrate ScyllaDB with Jaeger Server
      • Integrate ScyllaDB with MindsDB
    • Tutorials
  • ScyllaDB for Administrators
    • Administration Guide
    • Procedures
      • Cluster Management
      • Backup & Restore
      • Change Configuration
      • Maintenance
      • Best Practices
      • Benchmarking ScyllaDB
      • Migrate from Cassandra to ScyllaDB
      • Disable Housekeeping
    • Security
      • ScyllaDB Security Checklist
      • Enable Authentication
      • Enable and Disable Authentication Without Downtime
      • Creating a Custom Superuser
      • Generate a cqlshrc File
      • Reset Authenticator Password
      • Enable Authorization
      • Grant Authorization CQL Reference
      • Certificate-based Authentication
      • Role Based Access Control (RBAC)
      • ScyllaDB Auditing Guide
      • Encryption: Data in Transit Client to Node
      • Encryption: Data in Transit Node to Node
      • Generating a self-signed Certificate Chain Using openssl
      • Configure SaslauthdAuthenticator
      • Encryption at Rest
      • LDAP Authentication
      • LDAP Authorization (Role Management)
      • Software Bill Of Materials (SBOM)
    • Admin Tools
      • Nodetool Reference
      • CQLSh
      • Admin REST API
      • Tracing
      • ScyllaDB SStable
      • ScyllaDB Types
      • SSTableLoader
      • cassandra-stress
      • SSTabledump
      • SSTableMetadata
      • ScyllaDB Logs
      • Seastar Perftune
      • Virtual Tables
      • Reading mutation fragments
      • Maintenance socket
      • Maintenance mode
      • Task manager
    • Version Support Policy
    • ScyllaDB Monitoring Stack
    • ScyllaDB Operator
    • ScyllaDB Manager
    • Upgrade Procedures
      • About Upgrade
      • Upgrade Guides
    • System Configuration
      • System Configuration Guide
      • scylla.yaml
      • ScyllaDB Snitches
    • Benchmarking ScyllaDB
    • ScyllaDB Diagnostic Tools
  • ScyllaDB for Developers
    • Develop with ScyllaDB
    • Tutorials and Example Projects
    • Learn to Use ScyllaDB
    • ScyllaDB Alternator
    • ScyllaDB Drivers
      • ScyllaDB CQL Drivers
      • ScyllaDB DynamoDB Drivers
  • CQL Reference
    • CQLSh: the CQL shell
    • Appendices
    • Compaction
    • Consistency Levels
    • Consistency Level Calculator
    • Data Definition
    • Data Manipulation
      • SELECT
      • INSERT
      • UPDATE
      • DELETE
      • BATCH
    • Data Types
    • Definitions
    • Global Secondary Indexes
    • Expiring Data with Time to Live (TTL)
    • Functions
    • Wasm support for user-defined functions
    • JSON Support
    • Materialized Views
    • Non-Reserved CQL Keywords
    • Reserved CQL Keywords
    • DESCRIBE SCHEMA
    • Service Levels
    • ScyllaDB CQL Extensions
  • Features
    • Lightweight Transactions
    • Global Secondary Indexes
    • Local Secondary Indexes
    • Materialized Views
    • Counters
    • Change Data Capture
      • CDC Overview
      • The CDC Log Table
      • Basic operations in CDC
      • CDC Streams
      • CDC Stream Generations
      • Querying CDC Streams
      • Advanced column types
      • Preimages and postimages
      • Data Consistency in CDC
    • Workload Attributes
    • Workload Prioritization
  • ScyllaDB Architecture
    • Data Distribution with Tablets
    • ScyllaDB Ring Architecture
    • ScyllaDB Fault Tolerance
    • Consistency Level Console Demo
    • ScyllaDB Anti-Entropy
      • ScyllaDB Hinted Handoff
      • ScyllaDB Read Repair
      • ScyllaDB Repair
    • SSTable
      • ScyllaDB SSTable - 2.x
      • ScyllaDB SSTable - 3.x
    • Compaction Strategies
    • Raft Consensus Algorithm in ScyllaDB
    • Zero-token Nodes
  • Troubleshooting ScyllaDB
    • Errors and Support
      • Report a ScyllaDB problem
      • Error Messages
      • Change Log Level
    • ScyllaDB Startup
      • Ownership Problems
      • ScyllaDB will not Start
      • ScyllaDB Python Script broken
    • Upgrade
      • Inaccessible configuration files after ScyllaDB upgrade
    • Cluster and Node
      • Handling Node Failures
      • Failure to Add, Remove, or Replace a Node
      • Failed Decommission Problem
      • Cluster Timeouts
      • Node Joined With No Data
      • NullPointerException
      • Failed Schema Sync
    • Data Modeling
      • ScyllaDB Large Partitions Table
      • ScyllaDB Large Rows and Cells Table
      • Large Partitions Hunting
      • Failure to Update the Schema
    • 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
    • ScyllaDB Monitor and Manager
      • Manager and Monitoring integration
      • Manager lists healthy nodes as down
    • Installation and Removal
      • Removing ScyllaDB on Ubuntu breaks system packages
  • Knowledge Base
    • Upgrading from experimental CDC
    • Compaction
    • Consistency in ScyllaDB
    • 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 ScyllaDB and supporting services as a custom user:group
    • Customizing CPUSET
    • Decoding Stack Traces
    • Snapshots and Disk Utilization
    • DPDK mode
    • Debug your database with Flame Graphs
    • Efficient Tombstone Garbage Collection in ICS
    • How to Change gc_grace_seconds for a Table
    • Gossip in ScyllaDB
    • Increase Permission Cache to Avoid Non-paged Queries
    • How does ScyllaDB LWT Differ from Apache Cassandra ?
    • Map CPUs to ScyllaDB Shards
    • ScyllaDB Memory Usage
    • NTP Configuration for ScyllaDB
    • Updating the Mode in perftune.yaml After a ScyllaDB Upgrade
    • POSIX networking for ScyllaDB
    • ScyllaDB consistency quiz for administrators
    • Recreate RAID devices
    • How to Safely Increase the Replication Factor
    • ScyllaDB and Spark integration
    • Increase ScyllaDB resource limits over systemd
    • ScyllaDB Seed Nodes
    • How to Set up a Swap Space
    • ScyllaDB Snapshots
    • ScyllaDB 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
    • ScyllaDB Nodes are Unresponsive
    • Update a Primary Key
    • Using the perf utility with ScyllaDB
    • Configure ScyllaDB Networking with Multiple NIC/IP Combinations
  • Reference
    • AWS Images
    • Azure Images
    • GCP Images
    • Configuration Parameters
    • Glossary
    • Limits
    • ScyllaDB Enterprise vs. Open Source Matrix
    • API Reference (BETA)
    • Metrics (BETA)
  • ScyllaDB University
  • ScyllaDB FAQ
  • Alternator: DynamoDB API in Scylla
    • Getting Started With ScyllaDB Alternator
    • ScyllaDB Alternator for DynamoDB users
    • Alternator-specific APIs
Docs Tutorials University Contact Us About Us
© 2025, ScyllaDB. All rights reserved. | Terms of Service | Privacy Policy | ScyllaDB, and ScyllaDB Cloud, are registered trademarks of ScyllaDB, Inc.
Last updated on 09 Apr 2025.
Powered by Sphinx 7.4.7 & ScyllaDB Theme 1.8.6