Skip to content

[WIP] storage: proposal quota pool#13869

Closed
petermattis wants to merge 2 commits intocockroachdb:masterfrom
petermattis:pmattis/proposal-quota
Closed

[WIP] storage: proposal quota pool#13869
petermattis wants to merge 2 commits intocockroachdb:masterfrom
petermattis:pmattis/proposal-quota

Conversation

@petermattis
Copy link
Copy Markdown
Collaborator

@petermattis petermattis commented Mar 1, 2017

This change is Reviewable

@petermattis
Copy link
Copy Markdown
Collaborator Author

@spencerkimball Per our discussion yesterday. This isn't fully fleshed out right now, but gives us something concrete to argue over.

@petermattis petermattis force-pushed the pmattis/proposal-quota branch 2 times, most recently from dd3f1cf to 6e031a0 Compare March 1, 2017 12:56
@petermattis
Copy link
Copy Markdown
Collaborator Author

This seems to work in simple testing. A node crash will cause a 2 second blip of unavailability if the range is very active. In normal operation, we'll limit proposals to the speed of the slowest replica, but with a buffer to avoid imposing the latency of the slower replicas on the commit latency.

Cc @bdarnell

Copy link
Copy Markdown
Member

@spencerkimball spencerkimball left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall this is pretty simple, which is positive. As mentioned in the review comments, I think the biggest concern would be a replica coming back online and wreaking havoc.

My earlier thought to avoid the 2s timeout seems less useful now that I'm looking at this. It would just push the problem I was looking to solve elsewhere – we'd be relying on node liveness instead of this 2s timeout.

Need unittests too, obviously.

)

const (
leaderProposalQuota = 1000
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need something more thoughtful here?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably. We'd like something that avoided Raft log truncation. This achieves that for the workload I'm using which involves small requests. More directly estimating the affect on Raft log size would be nice, though I'm not sure how to achieve that.

type quotaPool struct {
c chan int64

mu sync.Mutex
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any reason not to use atomic primitives?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This code was lifted from grpc. I haven't thought through the difficulties with using atomics here.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yep, could be tricky. You'll have to scrutinize the code carefully.

}

func newQuotaPool(q int64) *quotaPool {
qb := &quotaPool{
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this qb, not qp?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cause I stole it from the grpc code base which uses the name qb. I'll change it eventually.

r.mu.proposalQuotaBase = r.mu.lastIndex
r.proposalQuota.reset(leaderProposalQuota)
} else {
r.proposalQuota.reset(followerProposalQuota)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do followers require a proposal quota?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So they can propose requests which get forwarded to the leader. For example, followers propose lease requests which is how they determine who holds the lease. We could probably change this so that we don't acquire proposal quota when a follower proposes a Raft command.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd just as soon eliminate the follower's use of a quota pool and the associated constant and just set the pool to nil on a leader -> follower transition, and allow followers to always propose and forward to the leader.

// Computed checksum at a snapshot UUID.
checksums map[uuid.UUID]replicaChecksum

proposalQuotaBase uint64
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

s/proposalQuotaBase/proposalQuotaBaseIndex/

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

if req.Message.Type == raftpb.MsgApp {
r.setEstimatedCommitIndexLocked(req.Message.Commit)
}
r.setLastActivityLocked(req.FromReplica.ReplicaID)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible for a hopelessly lagged replica to come back online and require a snapshot, but still report its log index and cause the quota pool to be immediately drained to a negative value?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, that's not possible. The purpose of Replica.mu.proposalQuotaBaseIndex is that the index used for quota calculations ratchets up. If a replica comes back on line and has a commit index before the quota base index, it will be ignored for quota purposes until it catches back up.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, right.

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, this was broken before, but should be fixed now.

for _, rep := range r.mu.state.Desc.Replicas {
// Only consider followers that we've received a message from in the last 2
// seconds.
const activeTime = 2 * time.Second
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably need to put this in StoreConfig.

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep. I'll add a TODO for now.

Copy link
Copy Markdown
Collaborator Author

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, unit tests are definitely needed before merging.

)

const (
leaderProposalQuota = 1000
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably. We'd like something that avoided Raft log truncation. This achieves that for the workload I'm using which involves small requests. More directly estimating the affect on Raft log size would be nice, though I'm not sure how to achieve that.

}

func newQuotaPool(q int64) *quotaPool {
qb := &quotaPool{
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cause I stole it from the grpc code base which uses the name qb. I'll change it eventually.

// Computed checksum at a snapshot UUID.
checksums map[uuid.UUID]replicaChecksum

proposalQuotaBase uint64
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

for _, rep := range r.mu.state.Desc.Replicas {
// Only consider followers that we've received a message from in the last 2
// seconds.
const activeTime = 2 * time.Second
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep. I'll add a TODO for now.

r.mu.proposalQuotaBase = r.mu.lastIndex
r.proposalQuota.reset(leaderProposalQuota)
} else {
r.proposalQuota.reset(followerProposalQuota)
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So they can propose requests which get forwarded to the leader. For example, followers propose lease requests which is how they determine who holds the lease. We could probably change this so that we don't acquire proposal quota when a follower proposes a Raft command.

if req.Message.Type == raftpb.MsgApp {
r.setEstimatedCommitIndexLocked(req.Message.Commit)
}
r.setLastActivityLocked(req.FromReplica.ReplicaID)
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, that's not possible. The purpose of Replica.mu.proposalQuotaBaseIndex is that the index used for quota calculations ratchets up. If a replica comes back on line and has a commit index before the quota base index, it will be ignored for quota purposes until it catches back up.

type quotaPool struct {
c chan int64

mu sync.Mutex
Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This code was lifted from grpc. I haven't thought through the difficulties with using atomics here.

@spencerkimball
Copy link
Copy Markdown
Member

Aside from unittests, LGTM

@petermattis petermattis force-pushed the pmattis/proposal-quota branch 2 times, most recently from ec6f53a to e5c6396 Compare March 1, 2017 19:20
@petermattis
Copy link
Copy Markdown
Collaborator Author

I reworked the handling of quotaPool so that it uses Replica.mu instead of its own internal lock and I changed the code so that there is only a non-nil quotaPool on the leader.

@petermattis
Copy link
Copy Markdown
Collaborator Author

Review status: 0 of 3 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending.


pkg/storage/replica.go, line 2802 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I'd just as soon eliminate the follower's use of a quota pool and the associated constant and just set the pool to nil on a leader -> follower transition, and allow followers to always propose and forward to the leader.

Done.


Comments from Reviewable

@petermattis
Copy link
Copy Markdown
Collaborator Author

Review status: 0 of 3 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending.


pkg/storage/quota_pool.go, line 18 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

yep, could be tricky. You'll have to scrutinize the code carefully.

Reworked to remove the mutex.


pkg/storage/quota_pool.go, line 23 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Cause I stole it from the grpc code base which uses the name qb. I'll change it eventually.

Done.


Comments from Reviewable

@petermattis petermattis force-pushed the pmattis/proposal-quota branch 2 times, most recently from 7044091 to fd1947d Compare March 1, 2017 19:44
@petermattis
Copy link
Copy Markdown
Collaborator Author

Review status: 0 of 3 files reviewed at latest revision, 7 unresolved discussions, some commit checks pending.


pkg/storage/quota_pool.go, line 18 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Reworked to remove the mutex.

Doh! Removing the mutex clearly wasn't safe. Reintroduced for now.


Comments from Reviewable

@petermattis petermattis force-pushed the pmattis/proposal-quota branch from fd1947d to 813d7bc Compare March 2, 2017 12:35
@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Mar 5, 2017

Reviewed 1 of 3 files at r1, 2 of 2 files at r2.
Review status: all files reviewed at latest revision, 16 unresolved discussions, some commit checks failed.


pkg/storage/quota_pool.go, line 75 at r2 (raw file):

		qp.c <- q
	} else {
		qp.quota = q

Is q always zero here (in which case this assignment is redundant) or are negative values allowed?


pkg/storage/quota_pool.go, line 101 at r2 (raw file):

// acquires acquires a single unit of quota from the pool. Safe for concurrent
// use.

Document what is required of the caller: if acquire() returns without error, the caller must call add(1) after doing its work.


pkg/storage/replica.go, line 825 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Yep. I'll add a TODO for now.

We have a lot of different failure detectors, with different thresholds and even different mechanisms. Can this piggyback on another (probably NodeLiveness) instead of introducing a new way for a node to be "down"?

A failing node already triggers a blip of up to 9s (?) for ranges it leads, so it doesn't seem bad to me to lengthen the impact of a failing node on ranges where it is a follower which are also maxing out their proposal quota.


pkg/storage/replica.go, line 812 at r2 (raw file):

		r.mu.lastActivity = make(map[roachpb.ReplicaID]time.Time)
	}
	r.mu.lastActivity[replicaID] = timeutil.Now()

We should also clear from this map at some point - at least when removing a replica, and maybe also empty the whole map when leadership changes hands?


pkg/storage/replica.go, line 825 at r2 (raw file):

}

func (r *Replica) refreshProposalQuotaLocked(newLeaderID roachpb.ReplicaID) {

s/refresh/update/?


pkg/storage/replica.go, line 840 at r2 (raw file):

		return
	}
	// We're the leader.

Add "still" to this comment to distinguish it from the early return above when we're becoming the leader.


pkg/storage/replica.go, line 842 at r2 (raw file):

	// We're the leader.

	// TODO(peter): Can we avoid retrieving the Raft status on every invocation?

I assume your concern is for the allocations this performs? We could probably add a method to query this without additional allocations. We could also maintain our own map by tracking MsgAppResps as they pass through the transport, but that probably duplicates too much raft logic.


pkg/storage/replica.go, line 844 at r2 (raw file):

	// TODO(peter): Can we avoid retrieving the Raft status on every invocation?
	status := r.raftStatusRLocked()
	// Find the minimum index that active followers have committed.

s/committed/acknowledged/


pkg/storage/replica.go, line 857 at r2 (raw file):

		}
		if progress, ok := status.Progress[uint64(rep.ReplicaID)]; ok {
			// Only consider follower's who are in advance of the quota base

s/'//


pkg/storage/replica.go, line 870 at r2 (raw file):

	if r.mu.proposalQuotaBaseIndex < minIndex {
		delta := int64(minIndex - r.mu.proposalQuotaBaseIndex)

Add a comment pointing out this subtlety: 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.


Comments from Reviewable

@petermattis petermattis force-pushed the pmattis/proposal-quota branch from 813d7bc to 7369586 Compare March 6, 2017 02:11
@petermattis
Copy link
Copy Markdown
Collaborator Author

I still need to add tests.


Review status: 1 of 3 files reviewed at latest revision, 16 unresolved discussions.


pkg/storage/quota_pool.go, line 75 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Is q always zero here (in which case this assignment is redundant) or are negative values allowed?

We always pass in a positive value to newQuotaPool. I can simplify this.


pkg/storage/quota_pool.go, line 101 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Document what is required of the caller: if acquire() returns without error, the caller must call add(1) after doing its work.

Done.


pkg/storage/replica.go, line 825 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

We have a lot of different failure detectors, with different thresholds and even different mechanisms. Can this piggyback on another (probably NodeLiveness) instead of introducing a new way for a node to be "down"?

A failing node already triggers a blip of up to 9s (?) for ranges it leads, so it doesn't seem bad to me to lengthen the impact of a failing node on ranges where it is a follower which are also maxing out their proposal quota.

I initially piggybacked this on node liveness, but then got worried that the node liveness ranges would have to be exempted in order to avoid deadlock. Seemed simpler to have a different mechanism here.


pkg/storage/replica.go, line 812 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

We should also clear from this map at some point - at least when removing a replica, and maybe also empty the whole map when leadership changes hands?

Done. We now only initialize this map on the leader and clear it when a node becomes a follower.


pkg/storage/replica.go, line 825 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/refresh/update/?

Done.


pkg/storage/replica.go, line 840 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Add "still" to this comment to distinguish it from the early return above when we're becoming the leader.

Done.


pkg/storage/replica.go, line 842 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I assume your concern is for the allocations this performs? We could probably add a method to query this without additional allocations. We could also maintain our own map by tracking MsgAppResps as they pass through the transport, but that probably duplicates too much raft logic.

Yeah, the allocation is the concern. It is a small concern, though. Trying to track this ourselves via MsgAppResp seems like overkill. I've enhanced the TODO comment.


pkg/storage/replica.go, line 844 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/committed/acknowledged/

Done.


pkg/storage/replica.go, line 857 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/'//

Done.


pkg/storage/replica.go, line 870 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Add a comment pointing out this subtlety: 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.

Done.


Comments from Reviewable

@petermattis
Copy link
Copy Markdown
Collaborator Author

Review status: 1 of 3 files reviewed at latest revision, 16 unresolved discussions, some commit checks pending.


pkg/storage/replica.go, line 825 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

I initially piggybacked this on node liveness, but then got worried that the node liveness ranges would have to be exempted in order to avoid deadlock. Seemed simpler to have a different mechanism here.

Rather than introducing a new configurable here, perhaps this time should be the same as the Raft election timeout. The only way for a leader to have not heard for a Replica for longer than the Raft election timeout is for the Range to be quiescent in which case there are no concerns about proposal quota.


Comments from Reviewable

@petermattis
Copy link
Copy Markdown
Collaborator Author

Review status: 1 of 3 files reviewed at latest revision, 16 unresolved discussions, some commit checks pending.


pkg/storage/replica.go, line 825 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Rather than introducing a new configurable here, perhaps this time should be the same as the Raft election timeout. The only way for a leader to have not heard for a Replica for longer than the Raft election timeout is for the Range to be quiescent in which case there are no concerns about proposal quota.

Err, The only way is not correct. The primary way? A leader might not have heard from a replica if that replica is on a down/partitioned node.


Comments from Reviewable

@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Mar 6, 2017

Reviewed 2 of 2 files at r3.
Review status: all files reviewed at latest revision, 8 unresolved discussions, some commit checks failed.


pkg/storage/replica.go, line 825 at r1 (raw file):

but then got worried that the node liveness ranges would have to be exempted in order to avoid deadlock

OK, what about RPC heartbeats then? I'd really like to avoid introducing a new notion of node activity, especially one with subtle interactions with quiescence and leadership changes.


pkg/storage/replica.go, line 812 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Done. We now only initialize this map on the leader and clear it when a node becomes a follower.

So a brand-new leader will see all the followers as inactive. That seems less than ideal, although since we're also resetting the quota to its maximum limit when becoming leader I don't see a specific problem with it.


Comments from Reviewable

@petermattis petermattis force-pushed the pmattis/proposal-quota branch from 7369586 to ddc6ef8 Compare March 6, 2017 17:48
@petermattis
Copy link
Copy Markdown
Collaborator Author

Review status: 1 of 2 files reviewed at latest revision, 8 unresolved discussions.


pkg/storage/replica.go, line 825 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

but then got worried that the node liveness ranges would have to be exempted in order to avoid deadlock

OK, what about RPC heartbeats then? I'd really like to avoid introducing a new notion of node activity, especially one with subtle interactions with quiescence and leadership changes.

My instinct is that the new mechanism would be less fragile than reusing an existing one like RPC heartbeats. Not sure why I feel that way. I'll think about this more. That said, I added another commit which replaces lastActivity with RPC connection health. The various lookups involved are mildly concerning for performance.


Comments from Reviewable

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
@petermattis petermattis force-pushed the pmattis/proposal-quota branch from ddc6ef8 to f16f2db Compare March 27, 2017 15:33
@knz knz added the do-not-merge bors won't merge a PR with this label. label Apr 25, 2017
@petermattis
Copy link
Copy Markdown
Collaborator Author

Cc @irfansharif

irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 9, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 11, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 15, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 15, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 15, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 15, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 15, 2017
Repurposing cockroachdb#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.
irfansharif added a commit to irfansharif/cockroach that referenced this pull request May 16, 2017
Repurposing cockroachdb#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.
@petermattis petermattis deleted the pmattis/proposal-quota branch May 30, 2017 15:44
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

do-not-merge bors won't merge a PR with this label.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants