Skip to content

kv: make disk I/O asynchronous with respect to Raft state machine #17500

@nvb

Description

@nvb

⚠️ EDIT: The original optimization proposed here was implemented in #38954. See #17500 (comment) for the remainder of this issue.

Raft, along with most consensus protocols in the Paxos family, distinguishes committed entries from executed entries. An entry is committed when the Raft protocol has completed and the entry has been sufficiently replicated and persisted in the Raft log. The entry is then executed when it is applied to a given replica's state machine. This distinction is important because in cases where the execution of an entry onto the state machine does not produce a result, it is not necessary to wait for execution before sending a commit notification to clients.

Currently in Cockroach, the proposing replica waits until command execution before responding to clients. However, changes made for PropEval KV assured that all MVCC related logic is made upstream of Raft and that by the time we reach entry execution, we're simply applying a WriteBatch to RockDB. While some of these execution steps can create ReplicaCorruptionErrors, I don't think it's necessary or possibly even correct that we attach these errors to the proposal response itself. This is because the entry has already been replicated through Raft, so a local ReplicaCorruptionError doesn't mean that all replicas are corrupted or that the command failed. In fact, after looking at the code I don't think that proposalResult needs anything populated at execution time for correctness. Because of this, I have a suspicion that all of the time spent in applyRaftCommand, including writing to RocksDB and performing stats computations, is unnecessary latency.

Preliminary results from a (very) rough draft of the change show a 2-3% improvement on average latency for the KV benchmark with a --batch size of 4:

Without change (batch=4):

_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)
  600.0s        0         597164          995.3     32.2     32.5     41.9     65.0    939.5

BenchmarkBlocks	  597164	   1004753.2 ns/op

With change (batch=4):

_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)
  600.0s        0         612652         1021.1     31.4     30.4     41.9     58.7    604.0

BenchmarkBlocks	  612652	    979351.4 ns/op

With a --batch size of 40, the results were even more pronounced:

Without change (batch=40):

_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)
  300.0s        0         616040         2053.4    134.0     54.5    151.0   2415.9  10200.5

BenchmarkBlocks	  616040	    486990.5 ns/op

With change (batch=40):

_elapsed___errors_____ops(total)___ops/sec(cum)__avg(ms)__p50(ms)__p95(ms)__p99(ms)_pMax(ms)
  300.0s        0         708040         2360.1    114.7     54.5    130.0   1140.9  10200.5

BenchmarkBlocks	  708040	    423713.8 ns/op

Note that while I think we can respond to clients sooner, I don't think we can pop the command out of the CommandQueue until after it has actually been executed because of how proposer evaluated kv works upstream of Raft.

@tschottdorf @bdarnell

Jira issue: CRDB-6037

Epic CRDB-22644

Metadata

Metadata

Assignees

Labels

A-kv-replicationRelating to Raft, consensus, and coordination.C-performancePerf of queries or internals. Solution not expected to change functional behavior.T-kvKV Team

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions