Skip to content

Commit

Permalink
Merge #38343
Browse files Browse the repository at this point in the history
38343: storage: introduce concurrent Raft proposal buffer r=tbg a=nvanbenschoten

This change introduces a new multi-producer, single-consumer buffer for Raft proposal ingestion into the Raft replication pipeline. This buffer becomes the new coordination point between "above Raft" goroutines, who have just finished evaluation and want to replicate a command, and a Replica's "below Raft" goroutine, which collects these commands and begins the replication process.

The structure improves upon the current approach to this interaction in three important ways. The first is that the structure supports concurrent insertion of proposals by multiple proposer goroutines. This significantly increases the amount of concurrency for non-conflicting writes within a single Range. The proposal buffer does this without exclusive locking using atomics to index into an array. This is complicated by the strong desire for proposals to be proposed in the same order that their MaxLeaseIndex is assigned. The buffer addresses this by selecting a slot in its array and selecting a MaxLeaseIndex for a proposal in a single atomic operation.

The second improvement is that the new structure allows RaftCommand marshaling to be lifted entirely out of any critical section. Previously, the allocation, marshaling, and encoding processes for a RaftCommand was performed under the exclusive Replica lock. Before 91abab1, there was even a second allocation and a copy under this lock. This locking interacted poorly with both "above Raft" processing (which repeatedly acquires a shared lock) and "below Raft" processing (which occasionally acquires an exclusive lock). The new concurrent Raft proposal buffer structure is able to push this allocation and marshaling completely outside of the exclusive or shared Replica lock. It does so despite the fact that the MaxLeaseIndex of the RaftCommand has not been assigned yet by splitting marshaling into two steps and using a new "footer" proto. The first step is to allocate and marshal the majority of the encoded Raft command outside of any lock. The second step is to marshal just the small "footer" proto with the MaxLeaseIndex field into the same byte slice, which has been pre-sized with a small amount of extra capacity, after the MaxLeaseIndex has been selected. This approach lifts a major expense out of the Replica mutex.

The final improvement is to increase the amount of batching performed between Raft proposals. This reduces the number of messages required to coordinate their replication throughout the entire replication pipeline. To start, batching allows multiple Raft entries to be sent in the same MsgApp from the leader to followers. Doing so then results in only a single MsgAppResp being sent for all of these entries back to the leader, instead of one per entry. Finally, a single MsgAppResp results in only a single empty MsgApp with the new commit index being sent from the leader to followers. All of this is made possible by `Step`ping the Raft `RawNode` with a `MsgProp` containing multiple entries instead of using the `Propose` API directly, which internally `Step`s the Raft `RawNode` with a `MsgProp` containing only one entry. Doing so demonstrated a very large improvement in `rafttoy` and is showing a similar win here. The proposal buffer provides a clean place to perform this batching, so this is a natural time to introduce it.

### Benchmark Results

```
name                             old ops/sec  new ops/sec  delta
kv95/seq=false/cores=16/nodes=3   67.5k ± 1%   67.2k ± 1%     ~     (p=0.421 n=5+5)
kv95/seq=false/cores=36/nodes=3    144k ± 1%    143k ± 1%     ~     (p=0.320 n=5+5)
kv0/seq=false/cores=16/nodes=3    41.2k ± 2%   42.3k ± 3%   +2.49%  (p=0.000 n=10+10)
kv0/seq=false/cores=36/nodes=3    66.8k ± 2%   69.1k ± 2%   +3.35%  (p=0.000 n=10+10)
kv95/seq=true/cores=16/nodes=3    59.3k ± 1%   62.1k ± 2%   +4.83%  (p=0.008 n=5+5)
kv95/seq=true/cores=36/nodes=3     100k ± 1%    125k ± 1%  +24.37%  (p=0.008 n=5+5)
kv0/seq=true/cores=16/nodes=3     16.1k ± 2%   21.8k ± 4%  +35.21%  (p=0.000 n=9+10)
kv0/seq=true/cores=36/nodes=3     18.4k ± 3%   24.8k ± 2%  +35.29%  (p=0.000 n=10+10)

name                             old p50(ms)  new p50(ms)  delta
kv95/seq=false/cores=16/nodes=3    0.70 ± 0%    0.70 ± 0%     ~     (all equal)
kv95/seq=false/cores=36/nodes=3    0.70 ± 0%    0.70 ± 0%     ~     (all equal)
kv0/seq=false/cores=16/nodes=3     2.86 ± 2%    2.80 ± 0%   -2.10%  (p=0.011 n=10+10)
kv0/seq=false/cores=36/nodes=3     3.87 ± 2%    3.80 ± 0%   -1.81%  (p=0.003 n=10+10)
kv95/seq=true/cores=16/nodes=3     0.70 ± 0%    0.70 ± 0%     ~     (all equal)
kv95/seq=true/cores=36/nodes=3     0.70 ± 0%    0.70 ± 0%     ~     (all equal)
kv0/seq=true/cores=16/nodes=3      7.97 ± 2%    5.86 ± 2%  -26.44%  (p=0.000 n=9+10)
kv0/seq=true/cores=36/nodes=3      15.7 ± 0%    11.7 ± 4%  -25.61%  (p=0.000 n=8+10)

name                             old p99(ms)  new p99(ms)  delta
kv95/seq=false/cores=16/nodes=3    2.90 ± 0%    2.94 ± 2%     ~     (p=0.444 n=5+5)
kv95/seq=false/cores=36/nodes=3    3.90 ± 0%    3.98 ± 3%     ~     (p=0.444 n=5+5)
kv0/seq=false/cores=16/nodes=3     8.90 ± 0%    8.40 ± 0%   -5.62%  (p=0.000 n=10+8)
kv0/seq=false/cores=36/nodes=3     11.0 ± 0%    10.4 ± 3%   -5.91%  (p=0.000 n=10+10)
kv95/seq=true/cores=16/nodes=3     4.50 ± 0%    3.18 ± 4%  -29.33%  (p=0.000 n=4+5)
kv95/seq=true/cores=36/nodes=3     11.2 ± 3%     4.7 ± 0%  -58.04%  (p=0.008 n=5+5)
kv0/seq=true/cores=16/nodes=3      11.5 ± 0%     9.4 ± 0%  -18.26%  (p=0.000 n=9+9)
kv0/seq=true/cores=36/nodes=3      19.9 ± 0%    15.3 ± 2%  -22.86%  (p=0.000 n=9+10)
```

As expected, the majority of the improvement from this change comes when writing to a single Range (i.e. a write hotspot). In those cases, this change (and those in the following two commits) improves performance by up to **35%**.

NOTE: the Raft proposal buffer hooks into the rest of the Storage package through a fairly small and well-defined interface. The primary reason for doing so was to make the structure easy to move to a `storage/replication` package if/when we move in that direction.

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
  • Loading branch information
craig[bot] and nvanbenschoten committed Jun 26, 2019
2 parents 0bd5edb + a428a67 commit f7a282a
Show file tree
Hide file tree
Showing 19 changed files with 1,545 additions and 768 deletions.
6 changes: 0 additions & 6 deletions pkg/storage/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2171,9 +2171,6 @@ func TestQuotaPool(t *testing.T) {
if qLen := leaderRepl.QuotaReleaseQueueLen(); qLen != 1 {
return errors.Errorf("expected 1 queued quota release, found: %d", qLen)
}
if cLen := leaderRepl.CommandSizesLen(); cLen != 0 {
return errors.Errorf("expected zero-length command sizes map, found %d", cLen)
}
return nil
})

Expand All @@ -2196,9 +2193,6 @@ func TestQuotaPool(t *testing.T) {
if qLen := leaderRepl.QuotaReleaseQueueLen(); qLen != 0 {
return errors.Errorf("expected no queued quota releases, found: %d", qLen)
}
if cLen := leaderRepl.CommandSizesLen(); cLen != 0 {
return errors.Errorf("expected zero-length command sizes map, found %d", cLen)
}
return nil
})

Expand Down
10 changes: 1 addition & 9 deletions pkg/storage/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/rditer"
"github.com/cockroachdb/cockroach/pkg/storage/storagebase"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -256,7 +255,7 @@ func (r *Replica) GetLastIndex() (uint64, error) {
func (r *Replica) LastAssignedLeaseIndex() uint64 {
r.mu.RLock()
defer r.mu.RUnlock()
return r.mu.lastAssignedLeaseIndex
return r.mu.proposalBuf.LastAssignedLeaseIndexRLocked()
}

// SetQuotaPool allows the caller to set a replica's quota pool initialized to
Expand All @@ -273,7 +272,6 @@ func (r *Replica) InitQuotaPool(quota int64) {
}
r.mu.proposalQuota = newQuotaPool(quota)
r.mu.quotaReleaseQueue = nil
r.mu.commandSizes = make(map[storagebase.CmdIDKey]int)
}

// QuotaAvailable returns the quota available in the replica's quota pool. Only
Expand All @@ -296,12 +294,6 @@ func (r *Replica) IsFollowerActive(ctx context.Context, followerID roachpb.Repli
return r.mu.lastUpdateTimes.isFollowerActive(ctx, followerID, timeutil.Now())
}

func (r *Replica) CommandSizesLen() int {
r.mu.Lock()
defer r.mu.Unlock()
return len(r.mu.commandSizes)
}

// GetTSCacheHighWater returns the high water mark of the replica's timestamp
// cache.
func (r *Replica) GetTSCacheHighWater() hlc.Timestamp {
Expand Down
30 changes: 13 additions & 17 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,8 +222,6 @@ type Replica struct {
mergeComplete chan struct{}
// The state of the Raft state machine.
state storagepb.ReplicaState
// Counter used for assigning lease indexes for proposals.
lastAssignedLeaseIndex uint64
// Last index/term persisted to the raft log (not necessarily
// committed). Note that lastTerm may be 0 (and thus invalid) even when
// lastIndex is known, in which case the term will have to be retrieved
Expand Down Expand Up @@ -282,6 +280,12 @@ type Replica struct {
minLeaseProposedTS hlc.Timestamp
// A pointer to the zone config for this replica.
zone *config.ZoneConfig
// proposalBuf buffers Raft commands as they are passed to the Raft
// replication subsystem. The buffer is populated by requests after
// evaluation and is consumed by the Raft processing thread. Once
// consumed, commands are proposed through Raft and moved to the
// proposals map.
proposalBuf propBuf
// proposals stores the Raft in-flight commands which originated at
// this Replica, i.e. all commands for which propose has been called,
// but which have not yet applied.
Expand Down Expand Up @@ -381,8 +385,6 @@ type Replica struct {
// newly recreated replica will have a complete range descriptor.
lastToReplica, lastFromReplica roachpb.ReplicaDescriptor

// submitProposalFn can be set to mock out the propose operation.
submitProposalFn func(*ProposalData) error
// Computed checksum at a snapshot UUID.
checksums map[uuid.UUID]ReplicaChecksum

Expand All @@ -396,16 +398,11 @@ type Replica struct {

proposalQuotaBaseIndex uint64

// For command size based allocations we keep track of the sizes of all
// in-flight commands.
commandSizes map[storagebase.CmdIDKey]int

// Once the leader observes a proposal come 'out of Raft', we consult
// the 'commandSizes' map to determine the size of the associated
// command and add it to a queue of quotas we have yet to release back
// to the quota pool. We only do so when all replicas have persisted
// the corresponding entry into their logs.
quotaReleaseQueue []int
// Once the leader observes a proposal come 'out of Raft', we add the
// size of the associated command to a queue of quotas we have yet to
// release back to the quota pool. We only do so when all replicas have
// persisted the corresponding entry into their logs.
quotaReleaseQueue []int64

// Counts calls to Replica.tick()
ticks int
Expand Down Expand Up @@ -586,9 +583,8 @@ func (r *Replica) cleanupFailedProposalLocked(p *ProposalData) {
// NB: We may be double free-ing here in cases where proposals are
// duplicated. To counter this our quota pool is capped at the initial
// quota size.
if cmdSize, ok := r.mu.commandSizes[p.idKey]; ok {
r.mu.proposalQuota.add(int64(cmdSize))
delete(r.mu.commandSizes, p.idKey)
if r.mu.proposalQuota != nil {
r.mu.proposalQuota.add(p.quotaSize)
}
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/replica_closedts.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,13 @@ import (
// closed timestamp tracker. This is called to emit an update about this
// replica in the absence of write activity.
func (r *Replica) EmitMLAI() {
r.mu.Lock()
lai := r.mu.lastAssignedLeaseIndex
r.mu.RLock()
lai := r.mu.proposalBuf.LastAssignedLeaseIndexRLocked()
if r.mu.state.LeaseAppliedIndex > lai {
lai = r.mu.state.LeaseAppliedIndex
}
epoch := r.mu.state.Lease.Epoch
r.mu.Unlock()
r.mu.RUnlock()

ctx := r.AnnotateCtx(context.Background())
_, untrack := r.store.cfg.ClosedTimestamp.Tracker.Track(ctx)
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/replica_destroy.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,7 @@ func (r *Replica) destroyRaftMuLocked(ctx context.Context, nextReplicaID roachpb

func (r *Replica) cancelPendingCommandsLocked() {
r.mu.AssertHeld()
r.mu.proposalBuf.FlushLockedWithoutProposing()
for _, p := range r.mu.proposals {
r.cleanupFailedProposalLocked(p)
// NB: each proposal needs its own version of the error (i.e. don't try to
Expand Down
1 change: 1 addition & 0 deletions pkg/storage/replica_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ func (r *Replica) initRaftMuLockedReplicaMuLocked(
// reloading the raft state below, it isn't safe to use the existing raft
// group.
r.mu.internalRaftGroup = nil
r.mu.proposalBuf.Init((*replicaProposer)(r))

var err error
if r.mu.state, err = r.mu.stateLoader.Load(ctx, r.store.Engine(), desc); err != nil {
Expand Down
12 changes: 12 additions & 0 deletions pkg/storage/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,18 @@ type ProposalData struct {
// reproposals its MaxLeaseIndex field is mutated.
command *storagepb.RaftCommand

// encodedCommand is the encoded Raft command, with an optional prefix
// containing the command ID.
encodedCommand []byte

// quotaSize is the encoded size of command that was used to acquire
// proposal quota. command.Size can change slightly as the object is
// mutated, so it's safer to record the exact value used here.
quotaSize int64

// tmpFooter is used to avoid an allocation.
tmpFooter storagepb.RaftCommandFooter

// endCmds.finish is called after command execution to update the
// timestamp cache & release latches.
endCmds *endCmds
Expand Down
Loading

0 comments on commit f7a282a

Please sign in to comment.