Skip to content

Commit

Permalink
storage: flow control throttling replica operations
Browse files Browse the repository at this point in the history
Repurposing #13869.

The leader maintains a pool of "proposal quota". Before proposing a Raft
command, we acquire 1 unit of proposal quota. When all of the healthy
followers have committed an entry, that unit of proposal quota is
returned to the pool. The proposal quota pool size is hard coded to 1000
which allows fairly deep pipelining of Raft commands.

We only consider followers that have "healthy" RPC connections when
determining if a unit of quota should be returned to the pool.
  • Loading branch information
irfansharif committed May 9, 2017
1 parent 97ff303 commit aabef51
Show file tree
Hide file tree
Showing 4 changed files with 294 additions and 0 deletions.
87 changes: 87 additions & 0 deletions pkg/storage/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1678,6 +1678,93 @@ func TestReplicateRemoveAndAdd(t *testing.T) {
testReplicaAddRemove(t, false)
}

// TestQuotaPool verifies that writes get throttled in the case where we have
// two fast moving replicas with sufficiently fast growing raft logs and a
// slower replica catching up. By throttling write throughput we avoid having
// to constantly catch up the slower node via snapshots. See #8659.
func TestQuotaPool(t *testing.T) {
defer leaktest.AfterTest(t)()

mtc := &multiTestContext{}
mtc.Start(t, 3)
defer mtc.Stop()

const rangeID = 1
const quota = 100
mtc.replicateRange(rangeID, 1, 2)

mtc.stores[0].SetRaftLogQueueActive(false)
mtc.stores[1].SetRaftLogQueueActive(false)
mtc.stores[2].SetRaftLogQueueActive(false)

// Heartbeats through the system go through the same code paths where we
// acquire quota for writes, to not have to account for this we pause
// heartbeats altogether.
for _, nl := range mtc.nodeLivenesses {
nl.PauseHeartbeat(true)
}

repl1, err := mtc.stores[0].GetReplica(rangeID)
if err != nil {
t.Fatal(err)
}

repl1.SetQuotaPool(quota)

repl3, err := mtc.stores[2].GetReplica(rangeID)
if err != nil {
t.Fatal(err)
}

// NB: See TestRaftBlockedReplica/#9914 for why we use a separate goroutine.
// We block the third replica.
var wg sync.WaitGroup
wg.Add(1)
go func() {
repl3.RaftLock()
wg.Done()
}()
wg.Wait()

// We can write up to 'quota' number of keys before writes get throttled.
// We verify this by writing this many keys and ensuring the next write is
// blocked.
incArgs := incrementArgs([]byte("k"), 1)
for i := 0; i < quota; i++ {
if _, err := client.SendWrapped(context.Background(), repl1, incArgs); err != nil {
t.Fatal(err)
}
}

ch := make(chan struct{}, 1)
go func() {
defer close(ch)
if _, err := client.SendWrapped(context.Background(), repl1, incArgs); err != nil {
t.Fatal(err)
}
ch <- struct{}{}
}()

ticker := time.After(15 * time.Millisecond)
select {
case <-ch:
t.Fatal(errors.New("write not throttled by the quota pool"))
case <-ticker:
}

mtc.waitForValues(roachpb.Key("k"), []int64{quota, quota, 0})

repl3.RaftUnlock()

mtc.waitForValues(roachpb.Key("k"), []int64{quota + 1, quota + 1, quota + 1})

select {
case <-ch:
default:
t.Fatal(errors.New("throttled write not unblocked"))
}
}

// TestRaftHeartbeats verifies that coalesced heartbeats are correctly
// suppressing elections in an idle cluster.
func TestRaftHeartbeats(t *testing.T) {
Expand Down
10 changes: 10 additions & 0 deletions pkg/storage/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,6 +254,16 @@ func (r *Replica) GetLease() (*roachpb.Lease, *roachpb.Lease) {
return r.getLease()
}

// SetQuotaPool allows the caller to set a replica's quota pool initialized to
// a given quota. Only safe to call on the leader replica.
func (r *Replica) SetQuotaPool(quota int64) {
r.mu.Lock()
defer r.mu.Unlock()

r.mu.proposalQuotaBaseIndex = r.mu.lastIndex
r.mu.proposalQuota = newQuotaPool(quota)
}

// GetTimestampCacheLowWater returns the timestamp cache low water mark.
func (r *Replica) GetTimestampCacheLowWater() hlc.Timestamp {
r.store.tsCacheMu.Lock()
Expand Down
108 changes: 108 additions & 0 deletions pkg/storage/quota_pool.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
// Copyright 2017 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

/*
*
* Copyright 2014, Google Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are
* met:
*
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above
* copyright notice, this list of conditions and the following disclaimer
* in the documentation and/or other materials provided with the
* distribution.
* * Neither the name of Google Inc. nor the names of its
* contributors may be used to endorse or promote products derived from
* this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
* "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
* LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
* A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
* OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
* SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
* LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*
*/

package storage

import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"golang.org/x/net/context"
)

const (
// TODO(peter): This setting needs additional thought. Should it be adjusted
// dynamically?
defaultProposalQuota = 1000
)

type quotaPool struct {
syncutil.Mutex

c chan int64
}

// newQuotaPool returns a new quota pool initialized with a given quota,
// newQuotaPool(0) disallowed.
func newQuotaPool(q int64) *quotaPool {
qp := &quotaPool{
c: make(chan int64, 1),
}
qp.c <- q
return qp
}

// add adds the specified quota back to the pool. Safe for concurrent use.
func (qp *quotaPool) add(v int64) {
if v == 0 {
return
}

var quota int64
qp.Lock()
select {
case n := <-qp.c:
quota = n
default:
}
quota += v
qp.c <- quota
qp.Unlock()
}

// acquire acquires a single unit of quota from the pool. On success, nil is
// returned and the caller must call add(1) or otherwise arrange for the quota
// to be returned to the pool. Safe for concurrent use.
func (qp *quotaPool) acquire(ctx context.Context) error {
select {
case <-ctx.Done():
return ctx.Err()
case n := <-qp.c:
if n > 1 {
qp.add(n - 1)
}
return nil
}
}
89 changes: 89 additions & 0 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -375,6 +375,9 @@ type Replica struct {
// Computed checksum at a snapshot UUID.
checksums map[uuid.UUID]replicaChecksum

proposalQuota *quotaPool
proposalQuotaBaseIndex uint64

// Counts calls to Replica.tick()
ticks int

Expand Down Expand Up @@ -784,6 +787,87 @@ func (r *Replica) setEstimatedCommitIndexLocked(commit uint64) {
}
}

func (r *Replica) maybeAcquireProposalQuota(ctx context.Context) error {
r.mu.RLock()
quota := r.mu.proposalQuota
r.mu.RUnlock()
// Quota acquisition only takes place on the leader node,
// r.mu.proposalQuota is set to nil if a node is a follower (see
// updateProposalQuotaLocked).
if quota == nil {
return nil
}
return quota.acquire(ctx)
}

func (r *Replica) updateProposalQuotaLocked(ctx context.Context, newLeaderID roachpb.ReplicaID) {
if r.mu.leaderID != newLeaderID {
if r.mu.replicaID == newLeaderID {
// We're becoming the leader.
r.mu.proposalQuotaBaseIndex = r.mu.lastIndex

// Raft may propose commands itself (specifically the empty commands when
// leadership changes), and these commands don't go through the code paths
// where we acquire quota from the pool. To offset this we reset
// the quota pool whenever leadership changes hands.
r.mu.proposalQuota = newQuotaPool(defaultProposalQuota)
} else {
// We're becoming a follower.
r.mu.proposalQuota = nil
}
return
} else if r.mu.proposalQuota == nil {
// We're a follower.
return
}
// We're still the leader.

// TODO(peter): Can we avoid retrieving the Raft status on every invocation
// in order to avoid the associated allocation? Tracking the progress
// ourselves via looking at MsgAppResp messages would be overkill. Perhaps
// another accessor on RawNode.
status := r.raftStatusRLocked()
// Find the minimum index that active followers have acknowledged.
minIndex := status.Commit

for _, rep := range r.mu.state.Desc.Replicas {
// Only consider followers that have "healthy" RPC connections. We don't
// use node liveness here as doing so could lead to deadlock unless we
// avoided enforcing proposal quota for node liveness ranges.
if r.store.cfg.Transport.resolver != nil {
addr, err := r.store.cfg.Transport.resolver(rep.NodeID)
if err != nil {
continue
}
if err := r.store.cfg.Transport.rpcContext.ConnHealth(addr.String()); err != nil {
continue
}
}
if progress, ok := status.Progress[uint64(rep.ReplicaID)]; ok {
// Only consider followers who are in advance of the quota base
// index. This prevents a follower from coming back online and preventing
// throughput to the range until it has caught up.
if progress.Match < r.mu.proposalQuotaBaseIndex {
continue
}
if progress.Match > 0 && progress.Match < minIndex {
minIndex = progress.Match
}
}
}

if r.mu.proposalQuotaBaseIndex < minIndex {
// We've persisted minIndex - r.mu.proposalQuotaBaseIndex entries to
// the raft log since last we checked, the delta which
// can be released back to the quota pool.
// Entries with an index less than minIndex have been persisted on all
// followers with "healthy" RPC connections.
delta := int64(minIndex - r.mu.proposalQuotaBaseIndex)
r.mu.proposalQuotaBaseIndex = minIndex
r.mu.proposalQuota.add(delta)
}
}

// getEstimatedBehindCountRLocked returns an estimate of how far this replica is
// behind. A return value of 0 indicates that the replica is up to date.
func (r *Replica) getEstimatedBehindCountRLocked(raftStatus *raft.Status) int64 {
Expand Down Expand Up @@ -2225,6 +2309,10 @@ func (r *Replica) tryExecuteWriteBatch(

log.Event(ctx, "raft")

if err := r.maybeAcquireProposalQuota(ctx); err != nil {
return nil, roachpb.NewError(err), proposalNoRetry
}

ch, tryAbandon, err := r.propose(ctx, lease, ba, endCmds, spans)
if err != nil {
return nil, roachpb.NewError(err), proposalNoRetry
Expand Down Expand Up @@ -2842,6 +2930,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
r.store.raftEntryCache.addEntries(r.RangeID, rd.Entries)
r.mu.lastIndex = lastIndex
r.mu.raftLogSize = raftLogSize
r.updateProposalQuotaLocked(ctx, leaderID)
r.mu.leaderID = leaderID
r.mu.Unlock()

Expand Down

0 comments on commit aabef51

Please sign in to comment.