Use of Time in Distributed Databases (part 4): Synchronized clocks in production databases

This is part 4 of our "Use of Time in Distributed Databases" series. In this post, we explore how synchronized physical clocks enhance production database systems.

Spanner

Google's Spanner (OSDI'12) implemented a novel approach to handling time in distributed database systems through its TrueTime API. TrueTime API provides time as an interval that is guaranteed to contain the actual time, maintained within about 6ms (this is 2012 published number which improved significantly since then) of uncertainty using GPS receivers and atomic clocks. This explicit handling of time uncertainty allows Spanner to provide strong consistency guarantees while operating at a global scale.

Spanner uses multi-version concurrency control (MVCC) and achieves external consistency (linearizability) for current transactions through techniques like "commit wait," where transactions wait out the uncertainty in their commit timestamps before making their writes visible. Spanner uses Paxos groups approach to shard (partition) data across up to hundreds of servers, but thanks to TrueTime clocks, the system is able to provide linearizability to transactions without having to coordinate across all shards.

The real power of TrueTime becomes apparent in its handling of lock-free snapshot reads (reads of past database states). Without TrueTime, capturing consistent snapshots across a distributed database would require extensively tracking and storing causality relationships between different versions of data across the system. By using precisely synchronized time as the basis for versioning, Spanner can easily identify globally consistent reads of the database at any point in the past simply by specifying a timestamp in a lock free manner.


CockroachDB

CockroachDB builds on concepts from Google's Spanner, using a distributed key-value store organized into ranges (approximately 64MB chunks)--with each range forming its own Raft consensus group.

The key innovation of CockroachDB lies in achieving strong consistency and geo-distribution without relying on tightly-synchronized atomic clocks. Instead, it uses NTP-synchronized clocks alongside hybrid logical clocks (HLCs) and intent locks to handle time uncertainty. Unlike Spanner, which uses TrueTime and "commit-wait" to handle clock uncertainty, CockroachDB refrains from commit-waits due to longer NTP uncertainty intervals. CockroachDB dynamically adjusts transaction timestamps upward when conflicts occur within the uncertainty window. This approach eliminates the need for tightly-synchronized clocks, but leads to more transaction aborts/restarts under high contention.


CockroachDB implements serializable isolation through a MVCC system. Serializable isolation guarantees transactions appear to execute in isolation, but strict serializability adds a real-time ordering constraint: if transaction T1 commits before T2 begins, T1's commit timestamp must precede T2's. With NTP-synchronized clocks, CockroachDB achieves serializability but may violate strict serializability due to clock skew between nodes.

This limitation is illustrated through a scenario with three transactions across two keys: when transaction T1 reads both keys, it may see T3's update to y but not T2's earlier update to x, violating strict serializability. While this execution is serializable (as the transactions could be ordered T3, T1, T2), it breaks the real-time ordering guarantee that strict serializability requires. CockroachDB's reliance on looser clock synchronization makes such guarantees impossible without significant performance penalties. Spanner avoids these issues through its TrueTime API and tightly synchronized clocks, as it can afford performing commit-wait (delaying transaction commits until the timestamp's upper bound has passed).

A recent paper investigated integrating high-precision clock synchronization into CockroachDB and the resulting impact on performance. This is a good read if you are interested in understanding CockroachDB's handling of time uncertainty in transaction processing.

YugabyteDB, which has a similar Raft-groups based shared nothing architecture, recently posted a great analysis of how to make AWS Time Sync Service with Precision Hardware Clock (with 50 microsecond clockbound) work with their database.


MongoDB

MongoDB's implementation of causal consistency also relies on Hybrid Logical Clocks (HLC). The system tracks ClusterTime as a tuple of Unix epoch seconds and an increment counter, allowing for fine-grained ordering within the same second.

This combines the benefits of physical and logical time, incrementing the logical clock only when operations are written to the primary node's replication log (op log). 

Causal consistency is implemented through sessions, where each operation carries the highest known operationTime, and nodes wait for their operation log to catch up to this time before processing requests. This approach enables cross-session and cross-client causal consistency while maintaining system availability during network partitions.

MongoDB uses ClusterTime also for running OCC general purpose transactions across multiple shards and provides snapshot isolation guarantee for the transactions. This takes longer to explain, so I will explain MongoDB's interactive/ACID distributed transaction protocol in another blog post. 


DynamoDB

DynamoDB's implementation of transactions uses a timestamp-based Bernstein-Goldman optimistic concurrency control (TSO) protocol from 1980, where transactions are assigned timestamps at the start to define their position in the serial order. Correctness does not depend on perfect clock synchronization --the timestamps can be treated as monotonically increasing logical timestamps since the protocol rejects smaller timestamps. Synchronized clocks help improve the performance of transactions by reducing spurious conflicts.

To maintain its promise of predictable performance at scale, DynamoDB implements one-shot transactions rather than interactive transactions, allowing it to avoid the complexities of lock management and recovery. For write transactions, the system uses a two-phase protocol where the transaction coordinator checks timestamps against item timestamps before committing. While better synchronized clocks result in more successful transactions (as real-time ordering aligns better with serialization order), the system includes safeguards against clocks that are too far ahead and isolates transaction managers with faulty clocks.

For read-only transactions, DynamoDB employs a two-phase protocol that avoids maintaining read timestamps on items, which would turn every read into a costly write operation. Instead, it uses log sequence numbers (LSNs) to detect concurrent modifications between the two phases.


Accord

Accord protocol (2023) aims to implement general-purpose transactions in Cassandra. The key innovation is the use of synchronized clocks and hybrid logical clocks (HLCs) to achieve fast, consistent transactions across distributed systems.

Accord is a leaderless consensus protocol aims to achieve optimal latency (one WAN round-trip) under normal conditions while maintaining strict-serializable isolation. It comes from the same family of solutions like EPaxos, Caesar, and Tempo. Improving on Tempo (which we had reviewed in detail here), Accord handles timestamp conflicts more efficiently by allowing for inconsistent but valid sets of dependencies between transactions, requiring only that all dependency sets include those that may be committed with a lower timestamp.

This time-based approach, combined with the Reorder Buffer and Fast Path Electorate concepts, helps achieve improve performance. Accord's reorder buffer uses knowledge of maximum clock skew between nodes and point-to-point latencies to ensure messages are processed in the correct timestamp order. The system buffers timestamp proposals for a period equal to the clock skew plus the longest point-to-point latency, ensuring that potentially conflicting messages have time to arrive before processing begins. Accord uses HLCs that augments timestamps with unique replica IDs to achive both temporal ordering and global uniqueness of transactions.

If you just want to focus on the abstract idea, Detock (SIGMOD 2023) presents a similar idea for serializing transactions using time. So yeah there is a hint of deterministic databases here. 


TIDB: snapshot isolation (Timestamp Oracle)

TiDB employs a centralized timestamp management system through its Placement Driver (PD), which hosts the timestamp oracle (TSO). The TSO generates globally unique timestamps that combine physical time (with millisecond accuracy) and logical time (using 18 bits), serving as transaction IDs. This hybrid logical clock approach ensures strict ordering of transactions while maintaining a close relationship with real time.

TiDB uses these timestamps to implement snapshot isolation (SI) through multi-version concurrency control (MVCC). Unlike CockroachDB which provides serializability, TiDB focuses on snapshot isolation and read committed isolation levels. The timestamps play a crucial role in transaction management, allowing the system to maintain consistent views of the data and protect against write-write conflicts. The system supports both optimistic and pessimistic concurrency control methods, with the optimistic approach providing better throughput in low-conflict situations.

An important limitation of TiDB's time-based approach is that its centralized TSO makes multi-region deployment challenging. While the system uses Raft for replication within a single region (organizing data into 96MB range shards they call "regions"), geo-distribution would require modifications to both the TSO and Placement Driver to handle cross-region timestamp coordination effectively.


Aurora Limitless

Aurora Limitless is not a different engine, but rather a managed sharding option built on top of Aurora (which we reviewed here). Instead of a single writer it allows for multiple writers by leveraging precise time synchronization to achieve scalable transactions. The system uses Amazon TimeSync, which provides microsecond-accurate time synchronization with bounded uncertainty (under 50 microseconds), combined with Hybrid Logical Clocks (HLC) to ensure consistent ordering of events across the distributed system.

Aurora Limitless modifies PostgreSQL's traditional snapshot isolation mechanism to work in a distributed environment by implementing snapshots "as of then" rather than "as of now." To handle potential issues with transaction visibility and commit timing, Aurora Limitless uses clever techniques like predicting commit completion times and setting commit timestamps slightly in the future. This approach helps maintain read performance by ensuring reads are never blocked by writes while still preserving consistency guarantees.

The architecture employs a two-phase commit (2PC) protocol with an interesting twist: transaction routers initially coordinate commits but hand off supervision to local shard coordinators for high availability. This is because we don't want to require high-availability at the router, and keep it soft-state. Shards already  have high availability built-in, so the router hands off the supervision 2PC completion to a shard leader, which is already replicated and highly available.  The system also addresses the challenge of "zombie shards" (nodes that don't know they've been failed over) through consistency leases that expire after a few seconds to prevent stale reads. 


Aurora DSQL

Aurora DSQL (2024) uses time as a fundamental mechanism for managing transaction consistency and isolation. Unlike previous Aurora databases, it employs a different, truly serverless architecture with two key services: an adjudicator service for concurrency control and a journal service for update durability.

The system uses timestamps in two crucial ways: first, when a transaction starts, it chooses a start time (Ts) for performing all reads against the multiversion concurrency control (MVCC) storage system. Second, when committing, it selects a commit time (Tc). These timestamps help maintain snapshot isolation by ensuring that conflicts are properly detected between transactions that overlap in time.

DSQL's commit process is built around temporal coordination. When a transaction wants to commit, it must verify that no other transaction has written to the same keys between its start time (Ts) and commit time (Tc). This verification is handled by the adjudicator service, which makes guarantees about transaction ordering based on timestamps. The adjudicator promises to never commit a transaction with an earlier timestamp once it has allowed a transaction to commit at a given time, establishing a temporal ordering of transactions. Each adjudicator has jurisdiction over specific key-ranges. Adjudicators don't use Paxos for leader-election/fault-tolerance, but rather they again leverage time/fencing during handovers of key-range responsibility.

The storage system uses timestamps to determine when it has a complete view of the data at a given point in time. To handle cases where the write rate might be low, the system implements a heartbeat protocol where adjudicators commit to moving their commit points forward in sync with the physical clock. This ensures that storage nodes can confidently serve reads as of a particular timestamp by knowing they have seen all transactions up to that point.

This time-based approach allows Aurora DSQL to achieve strong consistency reads and snapshot isolated transactions while maintaining high performance through optimistic concurrency control (OCC). By moving all coordination to commit time and using timestamps for ordering, the system minimizes the necessary coordination between machines, availability zones, and regions.

Comments

Popular posts from this blog

Hints for Distributed Systems Design

Learning about distributed systems: where to start?

Making database systems usable

Looming Liability Machines (LLMs)

Advice to the young

Foundational distributed systems papers

Linearizability: A Correctness Condition for Concurrent Objects

Understanding the Performance Implications of Storage-Disaggregated Databases

Designing Data Intensive Applications (DDIA) Book

Use of Time in Distributed Databases (part 2): Use of logical clocks in databases