CockroachDB physical cluster replication (PCR) continuously sends all data at the cluster level from a primary cluster to an independent standby cluster. Existing data and ongoing changes on the active primary cluster, which is serving application data, replicate asynchronously to the passive standby cluster.
You can fail over from the primary cluster to the standby cluster. This will stop the replication stream, reset the standby cluster to a point in time (in the past or future) where all ingested data is consistent, and make the standby ready to accept application traffic.
For a list of requirements for PCR, refer to the Before you begin section of the setup tutorial.
Cockroach Labs also has a logical data replication tool that continuously replicates tables between an active source CockroachDB cluster to an active destination CockroachDB cluster. Both source and destination can receive application reads and writes, and participate in bidirectional LDR for eventual consistency in the replicating tables.
Use cases
You can use PCR to:
- Meet your RTO (Recovery Time Objective) and RPO (Recovery Point Objective) requirements. PCR provides lower RTO and RPO than backup and restore.
- Automatically replicate everything in your primary cluster to recover quickly from a control plane or full cluster failure.
- Protect against region failure when you cannot use individual multi-region clusters—for example, if you have a two-datacenter architecture and do not have access to three regions; or, you need low-write latency in a single region. PCR allows for an active-passive (primary-standby) structure across two clusters with the passive cluster in a different region.
- Quickly recover from user error (for example, dropping a database) by failing over to a time in the near past.
- Create a blue-green deployment model by using the standby cluster for testing upgrades and hardware changes.
Features
- Asynchronous cluster-level replication: When you initiate a replication stream, it will replicate byte-for-byte all of the primary cluster's existing user data and associated metadata to the standby cluster asynchronously. From then on, it will continuously replicate the primary cluster's data and metadata to the standby cluster. PCR will automatically replicate changes related to operations such as schema changes, user and privilege modifications, and zone configuration updates without any manual work.
- Transactional consistency: Avoid conflicts in data after recovery; the replication completes to a transactionally consistent state.
- Improved RPO and RTO: Depending on workload and deployment configuration, replication lag between the primary and standby is generally in the tens-of-seconds range. The failover process from the primary cluster to the standby should typically happen within five minutes when completing a failover to the latest replicated time using LATEST.
- Failover to a timestamp in the past or the future: In the case of logical disasters or mistakes, you can fail over from the primary to the standby cluster to a timestamp in the past. This means that you can return the standby to a timestamp before the mistake was replicated to the standby. Furthermore, you can plan a failover by specifying a timestamp in the future.
- Fast failback: Switch back from the promoted standby cluster to the original primary cluster after a failover event by replicating net-new changes rather than fully replacing existing data for an initial scan.
- New in v24.3:
Read from standby cluster: You can configure PCR to allow SELECTqueries on the standby cluster. For more details, refer to Start a PCR stream with read from standby.
- Monitoring: To monitor the replication's initial progress, current status, and performance, you can use metrics available in the DB Console and Prometheus. For more details, refer to Physical Cluster Replication Monitoring.
Known limitations
- Physical cluster replication is supported in:
- CockroachDB self-hosted clusters on v23.2 or later. The primary cluster can be a new or existing cluster. The standby cluster must be a new cluster started with the --virtualized-emptyflag.
- CockroachDB Advanced clusters on v24.3 or later.
 
- CockroachDB self-hosted clusters on v23.2 or later. The primary cluster can be a new or existing cluster. The standby cluster must be a new cluster started with the 
- In CockroachDB self-hosted, the primary and standby clusters must have the same zone configurations in order to respect data placement configurations. 
- After a failover, there is no mechanism to stop applications from connecting to the original primary cluster. It is necessary to redirect application traffic manually, such as by using a network load balancer or adjusting DNS records. 
Frequent large schema changes or imports may cause a significant spike in replication lag.
Get started
This section is a quick overview of the initial requirements to start a replication stream.
For more comprehensive guides, refer to:
- Cluster Virtualization Overview: for information on enabling cluster virtualization, a requirement for setting up PCR.
- Set Up Physical Cluster Replication: for a tutorial on how to start a replication stream.
- Physical Cluster Replication Monitoring: for detail on metrics and observability into a replication stream.
- Fail Over from a Primary Cluster to a Standby Cluster: for a guide on how to complete a replication stream and fail over to the standby cluster.
- Technical Overview: to understand PCR in more depth before setup.
Manage replication in the SQL shell
To start, manage, and observe PCR, you can use the following SQL statements:
| Statement | Action | 
|---|---|
| CREATE VIRTUAL CLUSTER ... FROM REPLICATION OF ... | Start a replication stream. | 
| ALTER VIRTUAL CLUSTER ... PAUSE REPLICATION | Pause a running replication stream. | 
| ALTER VIRTUAL CLUSTER ... RESUME REPLICATION | Resume a paused replication stream. | 
| ALTER VIRTUAL CLUSTER ... START SERVICE SHARED | Initiate a failover. | 
| SHOW VIRTUAL CLUSTER | Show all virtual clusters. | 
| DROP VIRTUAL CLUSTER | Remove a virtual cluster. | 
Cluster versions and upgrades
The entire standby cluster must be at the same version as, or one version ahead of, the primary's virtual cluster.
When PCR is enabled, upgrade with the following procedure. This upgrades the standby cluster before the primary cluster. Within the primary and standby CockroachDB clusters, the system virtual cluster must be at a cluster version greater than or equal to the virtual cluster:
- Upgrade the binaries on the primary and standby clusters. Replace the binary on each node of the cluster and restart the node.
- Finalize the upgrade on the standby's system virtual cluster if auto-finalization is disabled.
- Finalize the upgrade on the primary's system virtual cluster if auto-finalization is disabled.
- Finalize the upgrade on the standby's virtual cluster.
- Finalize the upgrade on the primary's virtual cluster.
Demo video
Learn how to use PCR to meet your RTO and RPO requirements with the following demo: