Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

storage: divergent entry application can lead to request stalls on Range splits #31330

Closed
nvanbenschoten opened this issue Oct 12, 2018 · 16 comments · Fixed by #31568
Closed
Assignees
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-performance Perf of queries or internals. Solution not expected to change functional behavior.

Comments

@nvanbenschoten
Copy link
Member

Most of this is taken from #30064 (comment). See that comment and its responses for a discussion on what's happening and why. The summary is that all followers in a Range can fall behind their leader with regard to the entries that they are applying at a given point in time. This separation can grow to the order of minutes. When this happens, splits will be applied on the leader minutes before they are applied on the followers, resulting in new Ranges that can't acheive quorum for minutes. Requests that are sent to these new Ranges stall for this period.

To reproduce:

  1. run workload tpcc init --warehouses=1000 against a 3-node cluster without nobarrier
  2. hook up request tracing
  3. watch for requests that take between 40 and 80 seconds
@nvanbenschoten nvanbenschoten added C-performance Perf of queries or internals. Solution not expected to change functional behavior. A-kv-replication Relating to Raft, consensus, and coordination. labels Oct 12, 2018
@nvanbenschoten nvanbenschoten self-assigned this Oct 12, 2018
@petermattis
Copy link
Collaborator

The summary is that all followers in a Range can fall behind their leader with regard to the entries that they are applying at a given point in time.

It's surprising that this can happen, though I can't think of anything that currently prevents it. Do you have any theory as to why the followers aren't keeping up with applying Raft commands? The follower still has the commands in its log. Are we not properly draining the log on followers? etcd-io/etcd#9982 added pagination to committed entries. Are we properly re-enqueuing the replica on the Raft scheduler when we only process a portion of the Raft log?

Cc @bdarnell

@tbg
Copy link
Member

tbg commented Oct 15, 2018

I know that TPCC sees the pagination fire at least on the districts table. I remember that from #28918.

@nvanbenschoten
Copy link
Member Author

etcd-io/etcd#9982 added pagination to committed entries.

I explored that change when looking into this. My theory was that if leaders and followers repeatedly hit a different code path in raftLog.slice for whatever reason, we could see followers slowly fall behind. I discounted it when seeing that the HardState.Commit index on followers was also falling behind. That's because I didn't notice this line, which was removed recently in etcd-io/etcd@7a8ab37. However, it turns out that we never actually bumped our vendored version of etcd/raft to include that change, and I just confirmed that the version I was running had the suspicious line. I think this could explain what I saw.

I'm going to re-run this with etcd-io/etcd@7a8ab37 and see if I can still reproduce.

@nvanbenschoten
Copy link
Member Author

I tested again with etcd-io/etcd@7a8ab37. This time, things made more sense. I still saw the divergent Raft applications between the leader and the followers but no longer saw the divergent commit indexes that I had before.

My next step is to crank up the MaxSizePerMsg config and see if that makes a difference. I can see that almost all Raft ready processing loops apply only a single Raft entry, so I suspect this is causing issues.

Are we properly re-enqueuing the replica on the Raft scheduler when we only process a portion of the Raft log?

That's a great question. I suspect that we're not. With the added knowledge that we're likely running into the MaxSizePerMsg on every Raft ready iteration, this could explain the divergence. We expect the leader to perform a greater number of Raft ready iterations than the followers, so it's very possible that this alone is what allows it to get ahead.

FWIW, we can detect this case with etcd-io/etcd@7a8ab37 merged by comparing rd.CommittedEntries[len(rd.CommittedEntries)-1].Index against rd.HardState.Commit.

@nvanbenschoten
Copy link
Member Author

That did the trick! Nice call @petermattis.

To summarize: when a Range is under a large amount of continuous load, it's possible that every Raft ready iteration it performs hits the pagination limit introduced in etcd-io/etcd#9982. If we don't detect these truncated Raft Ready structs and proactively requeue the Range to be processed again immediately then we fall into a situation where whichever peer runs through the most Raft ready iterations ends up applying the most entries. Naturally, the leader is the one who processes Raft the most because it's getting messages from both followers, so it pulls ahead.

I have two take-away questions from this, both for @bdarnell:

  1. why did we overload the meaning of MaxSizePerMsg? The value we default it to (16KB) makes sense for MsgApps, but seems pretty small for CommittedEntries that we plan to apply immediately. Was it just a matter of backwards compatability? Should we split them up?
  2. Detecting pagination is pretty delicate right now. How does a rd.CommittedEntriesPaginated flag sound? We can then check for this in handleRaftReadyRaftMuLocked and re-enqueue the Range to be processed again if necessary.

@petermattis
Copy link
Collaborator

How are you detecting that you have a truncated ready? I was thinking that handleRaftReadyRaftMuLocked could return a boolean to indicate the range should be requeued. Something like:

	var requeue bool
	if err := r.withRaftGroup(true, func(raftGroup *raft.RawNode) (bool, error) {
		raftGroup.Advance(rd)
		requeue = raftGroup.HasReady()
		return true, nil
	}); err != nil {
		...
	}

There is probably a cleaner way to do this, but you get the idea.

@nvanbenschoten
Copy link
Member Author

I was comparing rd.CommittedEntries[len(rd.CommittedEntries)-1].Index against rd.HardState.Commit, but I think that misses the case where the HardState is empty, which might actually be very common when a Range is requeued before any new messages arrive. That's what got me thinking about an rd.CommittedEntriesPaginated flag. Your approach should also work and doesn't require any etcd/raft changes, so I like it better. I'll try it out.

My plan was just to re-enqueue the Range update check again directly with the scheduler in these cases. That will be a little more expensive than looping directly above handleRaftReadyRaftMuLocked, but the code is cleaner and it also has the nice side effect of avoiding starvation. If that's too expensive to justify then we may be better off asking why the pagination size is so small.

@nvanbenschoten
Copy link
Member Author

So the change is as simple as:

diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go
index c6142df..9cd6f4d 100644
--- a/pkg/storage/replica.go
+++ b/pkg/storage/replica.go
@@ -4444,6 +4445,9 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
        const expl = "during advance"
        if err := r.withRaftGroup(true, func(raftGroup *raft.RawNode) (bool, error) {
                raftGroup.Advance(rd)
+               if raftGroup.HasReady() {
+                       r.store.enqueueRaftUpdateCheck(r.RangeID)
+               }
                return true, nil
        }); err != nil {
                return stats, expl, errors.Wrap(err, expl)

@bdarnell
Copy link
Contributor

why did we overload the meaning of MaxSizePerMsg? The value we default it to (16KB) makes sense for MsgApps, but seems pretty small for CommittedEntries that we plan to apply immediately. Was it just a matter of backwards compatability? Should we split them up?

When I made the raft change I was thinking MaxSizePerMsg was effectively "the amount of data we're willing to load into memory at once", and it would be nice to have one setting for that instead of multiplying them. But MaxSizePerMsg was changed in #10929 from 1MB to 16KB (I think in an attempt to reduce the time spent blocking to process an individual message, but these values seem awfully low to me today. Maybe we should revisit)

I think we should split things up to give CommittedEntries a separate limit (64MB?)

@nvanbenschoten
Copy link
Member Author

Yeah, I think we need to dramatically increase the CommittedEntries size limit. 64MB sounds good. Or we can rip it out after we have etcd-io/etcd#10167, which will limit the uncommitted entry size to 2MB. Once we do that, I think the patch in #31330 (comment) should be sufficient.

I tested this out on a single-node cluster on my Mac using the COCKROACH_RAFT_MAX_SIZE_PER_MSG env var. By setting this much higher, I was able to speed up workload init tpcc --warehouses=1 by a little over 20% (52s vs 41s). I'm also seeing what looks to be a 4-5% throughput increase on kv0 --max-block-bytes=1024 with this setting set much higher. In some senses, these are the best case scenarios for this change because they're writing a lot of data to a few ranges, but I wonder if this is hurting other write-heavy workloads as well.

@bdarnell
Copy link
Contributor

etcd-io/etcd#10167 limits the portion of the log that is not committed anywhere to 2MB. We still need separate pagination for CommittedEntries for followers who may be catching up (the original motivation was a follower that had a GB of logs to catch up on, even though those logs had been committed and applied on the other two nodes).

@nvanbenschoten
Copy link
Member Author

That makes sense. For now, I'll push to get the fix proposed in this issue into 2.1.1. Adding a new limit to etcd/raft is just a perf optimization, so it's not quite as urgent.

@tbg
Copy link
Member

tbg commented Oct 17, 2018

Good to finally see this coming to a close. BTW, were there any metrics that caught this? RaftLogFollowerBehindCount should've, right?

@nvanbenschoten
Copy link
Member Author

I don't think RaftLogFollowerBehindCount would have caught this. That's based on followers' match indexes, which wouldn't have reflected the divergence.

@tbg
Copy link
Member

tbg commented Oct 17, 2018

Ah, right. Should there be a new metric for this? Or a one-off check somewhere cheap that at least logs a warning when we're obviously farther behind than we'd like? This seems like one of the problems that will just go away when you fix it, but still.

@nvanbenschoten
Copy link
Member Author

Yeah, I'm not sure how useful that would be. Once we fix this, we should get back to the previous state of always applying all committed entries immediately on all followers, at which point a new metric wouldn't show anything useful. Of course, I hope we can move to a world where it would be absolutely essential - #17500.

nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Oct 18, 2018
Fixes cockroachdb#31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See cockroachdb#31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Oct 18, 2018
Fixes cockroachdb#31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See cockroachdb#31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Oct 18, 2018
Fixes cockroachdb#31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See cockroachdb#31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.
craig bot pushed a commit that referenced this issue Oct 18, 2018
31568: storage: re-enqueue Raft groups on paginated application r=nvanbenschoten a=nvanbenschoten

Fixes #31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See #31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.

Co-authored-by: Nathan VanBenschoten <[email protected]>
@craig craig bot closed this as completed in #31568 Oct 18, 2018
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Oct 19, 2018
Fixes cockroachdb#31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See cockroachdb#31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.
tbg pushed a commit to nvanbenschoten/cockroach that referenced this issue Oct 23, 2018
Fixes cockroachdb#31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See cockroachdb#31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.
nvanbenschoten added a commit to nvanbenschoten/cockroach that referenced this issue Nov 26, 2018
Fixes cockroachdb#31330.

This change re-enqueues Raft groups for processing immediately if they
still have more to do after a Raft ready iteration. This comes up in
practice when a Range has sufficient load to force Raft application
pagination. See cockroachdb#31330 for a discussion on the symptoms this can
cause.

Release note (bug fix): Fix bug where Raft followers could fall behind
leaders will entry application, causing stalls during splits.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-kv-replication Relating to Raft, consensus, and coordination. C-performance Perf of queries or internals. Solution not expected to change functional behavior.
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants