Skip to content

Commit

Permalink
[WIP] storage: proposal quota pool
Browse files Browse the repository at this point in the history
The leader maintains a pool of "proposal quota". Before proposing a Raft
command, we acquire 1 unit of proposal quota. When all of the active
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 "active" followers when determining if a unit of quota
should be returned to the pool. An active follower is one we've received
any type of message from in the past 2 seconds.

See cockroachdb#8659
  • Loading branch information
petermattis committed Mar 6, 2017
1 parent 3ba5ba3 commit 7822735
Show file tree
Hide file tree
Showing 3 changed files with 197 additions and 0 deletions.
109 changes: 109 additions & 0 deletions pkg/storage/quota_pool.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
// 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 {
c chan int64

mu syncutil.Mutex
quota int64
}

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) {
qp.mu.Lock()
defer qp.mu.Unlock()
select {
case n := <-qp.c:
qp.quota += n
default:
}
qp.quota += v
if qp.quota <= 0 {
return
}
select {
case qp.c <- qp.quota:
qp.quota = 0
default:
}
}

// 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
}
}
87 changes: 87 additions & 0 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -420,11 +420,16 @@ type Replica struct {
// newly recreated replica will have a complete range descriptor.
lastToReplica, lastFromReplica roachpb.ReplicaDescriptor

lastActivity map[roachpb.ReplicaID]time.Time

// 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

proposalQuota *quotaPool
proposalQuotaBaseIndex uint64

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

Expand Down Expand Up @@ -800,6 +805,83 @@ func (r *Replica) setEstimatedCommitIndexLocked(commit uint64) {
}
}

func (r *Replica) setLastActivityLocked(replicaID roachpb.ReplicaID) {
if r.mu.lastActivity != nil {
r.mu.lastActivity[replicaID] = timeutil.Now()
}
}

func (r *Replica) maybeAcquireProposalQuota(ctx context.Context) error {
r.mu.RLock()
quota := r.mu.proposalQuota
r.mu.RUnlock()
if quota == nil {
return nil
}
return quota.acquire(ctx)
}

func (r *Replica) updateProposalQuotaLocked(newLeaderID roachpb.ReplicaID) {
if r.mu.leaderID != newLeaderID {
if r.mu.replicaID == newLeaderID {
// We're becoming the leader.
r.mu.proposalQuotaBaseIndex = r.mu.lastIndex
r.mu.proposalQuota = newQuotaPool(defaultProposalQuota)
r.mu.lastActivity = make(map[roachpb.ReplicaID]time.Time)
} else {
// We're becoming a follower.
r.mu.proposalQuota = nil
r.mu.lastActivity = 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
now := timeutil.Now()
for _, rep := range r.mu.state.Desc.Replicas {
// Only consider followers that we've received a message from in the last 2
// seconds.
//
// TODO(peter): Put this in StoreConfig.
const activeTime = 2 * time.Second
if now.Sub(r.mu.lastActivity[rep.ReplicaID]) > activeTime {
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 && minIndex > progress.Match {
minIndex = progress.Match
}
}
}

if r.mu.proposalQuotaBaseIndex < minIndex {
// 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. We avoid releasing quota here that
// we never acquired by resetting the quota pool whenever leadership
// changes hands.
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 @@ -2179,6 +2261,10 @@ func (r *Replica) tryAddWriteCmd(

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)
if err != nil {
return nil, roachpb.NewError(err), proposalNoRetry
Expand Down Expand Up @@ -2749,6 +2835,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
r.store.raftEntryCache.addEntries(r.RangeID, rd.Entries)
r.mu.lastIndex = lastIndex
r.mu.raftLogSize = raftLogSize
r.updateProposalQuotaLocked(leaderID)
r.mu.leaderID = leaderID
r.mu.Unlock()

Expand Down
1 change: 1 addition & 0 deletions pkg/storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -3096,6 +3096,7 @@ func (s *Store) processRaftRequest(
if req.Message.Type == raftpb.MsgApp {
r.setEstimatedCommitIndexLocked(req.Message.Commit)
}
r.setLastActivityLocked(req.FromReplica.ReplicaID)
return false, /* !unquiesceAndWakeLeader */
raftGroup.Step(req.Message)
}); err != nil {
Expand Down

0 comments on commit 7822735

Please sign in to comment.