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: release quota on failed Raft proposals #38632

Merged

Conversation

nvanbenschoten
Copy link
Member

@nvanbenschoten nvanbenschoten commented Jul 3, 2019

Fixes #34180.
Fixes #35493.
Fixes #36983.
Fixes #37108.
Fixes #37371.
Fixes #37384.
Fixes #37551.
Fixes #37879.
Fixes #38095.
Fixes #38131.
Fixes #38136.
Fixes #38549.
Fixes #38552.
Fixes #38555.
Fixes #38560.
Fixes #38562.
Fixes #38563.
Fixes #38569.
Fixes #38578.
Fixes #38600.

A lot of the early issues fixed by this had previous failures, but nothing very recent or actionable. I think it's worth closing them now that they should be fixed in the short term.

This fixes a bug introduced in 1ff3556 where Raft proposal quota is no longer released when Replica.propose fails. This used to happen here, but that code was accidentally lost in the rewrite.

I tracked this down by running a series of import/tpch/nodes=4 and scrub/all-checks/tpcc/w=100 roachtests. About half the time, the import would stall after a few hours and the roachtest health reports would start logging lines like: n1/s1 2.00 metrics requests.slow.latch. I tracked the stalled latch acquisition to a stalled proposal quota acquisition by a conflicting command. The range debug page showed the following:

Screenshot_2019-07-01 r56 Range Debug Cockroach Console

We see that the Leaseholder of the Range has no pending commands but also no available proposal quota. This indicates a proposal quota leak, which led to me finding the lost release in this error case.

The (now confirmed) theory for what went wrong in these roachtests is that they are performing imports, which generate a large number of AddSSTRequests. These requests are typically larger than the available proposal quota for a range, meaning that they request all of its available quota. The effect of this is that if even a single byte of quota is leaked, the entire range will seize up and stall when an AddSSTRequests is issued. Instrumentation revealed that a ChangeReplicas request with a quota size equal to the leaked amount was failing due to the error:

received invalid ChangeReplicasTrigger REMOVE_REPLICA((n3,s3):3): updated=[(n1,s1):1 (n4,s4):2 (n2,s2):4] next=5 to remove self (leaseholder)

Because of the missing error handling, this quota was not being released back into the pool, causing future requests to get stuck indefinitely waiting for leaked quota, stalling the entire import.

@nvanbenschoten nvanbenschoten requested review from tbg and a team July 3, 2019 00:08
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@nvanbenschoten nvanbenschoten changed the title storage: release quota on failed Raft proposal storage: release quota on failed Raft proposals Jul 3, 2019
Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

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

:lgtm: thanks for tracking this down, my only substantial comment is about where to defer cleanupFailedProposal. Looking forward to a flurry of close notifications in my inbox.

A for of the early issues

Something's off there.

Reviewed 2 of 2 files at r1, 1 of 1 files at r2, 1 of 1 files at r3, 3 of 3 files at r4, 4 of 4 files at r5.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/storage/replica_proposal_buf.go, line 417 at r4 (raw file):

proposals,


pkg/storage/replica_raft.go, line 239 at r5 (raw file):

This ensures


pkg/storage/replica_raft.go, line 241 at r5 (raw file):

	// proposal buffer successfully. This ensure that we always release any
	// quota that we acquire.
	defer func() {

Is this in the right place? I'd have thought this would pair more naturally somewhere around

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

i.e. just after acquiring the quota. As written, we have to worry about any early returns between acquiring the quota and calling into propose. For example, and this doesn't matter in production but demonstrates the point, a TestingProposalFilter right now will leak any acquired quota:

if pErr := filter(filterArgs); pErr != nil {
return nil, nil, 0, pErr
}
}
maxLeaseIndex, pErr := r.propose(ctx, proposal)

This avoids confusing arithmetic in a few spots.

Release note: None
This commit makes propBuf.FlushLockedWithRaftGroup more careful about accidentally
dropping proposals when flushing its array. In most cases, any error will result
in a log.Fatal call further up the stack anyway, but it's good to be careful about
this stuff or we risk leaking proposals. We now make sure to register all proposals
with the proposer after an error, even if though we stop attempting to propose entries
with the Raft group.

Release note: None
Fixes cockroachdb#34180.
Fixes cockroachdb#35493.
Fixes cockroachdb#36983.
Fixes cockroachdb#37108.
Fixes cockroachdb#37371.
Fixes cockroachdb#37384.
Fixes cockroachdb#37551.
Fixes cockroachdb#37879.
Fixes cockroachdb#38095.
Fixes cockroachdb#38131.
Fixes cockroachdb#38136.
Fixes cockroachdb#38549.
Fixes cockroachdb#38552.
Fixes cockroachdb#38555.
Fixes cockroachdb#38560.
Fixes cockroachdb#38562.
Fixes cockroachdb#38563.
Fixes cockroachdb#38569.
Fixes cockroachdb#38578.
Fixes cockroachdb#38600.

_A lot of the early issues fixed by this had previous failures, but nothing
very recent or actionable. I think it's worth closing them now that they
should be fixed in the short term._

This fixes a bug introduced in 1ff3556 where Raft proposal quota is
no longer released when Replica.propose fails. This used to happen
[here](cockroachdb@1ff3556#diff-4315c7ebf8b8bf7bda469e1e7be82690L316),
but that code was accidentally lost in the rewrite.

I tracked this down by running a series of `import/tpch/nodes=4` and
`scrub/all-checks/tpcc/w=100` roachtests. About half the time, the
import would stall after a few hours and the roachtest health reports
would start logging lines like: `n1/s1  2.00  metrics  requests.slow.latch`.
I tracked the stalled latch acquisition to a stalled proposal quota acquisition
by a conflicting command. The range debug page showed the following:

<image>

We see that the leaseholder of the Range has no pending commands
but also no available proposal quota. This indicates a proposal
quota leak, which led to me finding the lost release in this
error case.

The (now confirmed) theory for what went wrong in these roachtests is that
they are performing imports, which generate a large number of AddSSTRequests.
These requests are typically larger than the available proposal quota
for a range, meaning that they request all of its available quota. The
effect of this is that if even a single byte of quota is leaked, the entire
range will seize up and stall when an AddSSTRequests is issued.
Instrumentation revealed that a ChangeReplicas request with a quota size
equal to the leaked amount was failing due to the error:
```
received invalid ChangeReplicasTrigger REMOVE_REPLICA((n3,s3):3): updated=[(n1,s1):1 (n4,s4):2 (n2,s2):4] next=5 to remove self (leaseholder)
```
Because of the missing error handling, this quota was not being released back
into the pool, causing future requests to get stuck indefinitely waiting for
leaked quota, stalling the entire import.

Release note: None
Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

Something's off there.

Done.

bors r=tbg

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @tbg)


pkg/storage/replica_proposal_buf.go, line 417 at r4 (raw file):

Previously, tbg (Tobias Grieger) wrote…

proposals,

Done.


pkg/storage/replica_raft.go, line 239 at r5 (raw file):

Previously, tbg (Tobias Grieger) wrote…

This ensures

Done.


pkg/storage/replica_raft.go, line 241 at r5 (raw file):

Previously, tbg (Tobias Grieger) wrote…

Is this in the right place? I'd have thought this would pair more naturally somewhere around

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

i.e. just after acquiring the quota. As written, we have to worry about any early returns between acquiring the quota and calling into propose. For example, and this doesn't matter in production but demonstrates the point, a TestingProposalFilter right now will leak any acquired quota:

if pErr := filter(filterArgs); pErr != nil {
return nil, nil, 0, pErr
}
}
maxLeaseIndex, pErr := r.propose(ctx, proposal)

Done.

craig bot pushed a commit that referenced this pull request Jul 3, 2019
38632: storage: release quota on failed Raft proposals r=tbg a=nvanbenschoten

Fixes #34180.
Fixes #35493.
Fixes #36983.
Fixes #37108.
Fixes #37371.
Fixes #37384.
Fixes #37551.
Fixes #37879.
Fixes #38095.
Fixes #38131.
Fixes #38136.
Fixes #38549.
Fixes #38552.
Fixes #38555.
Fixes #38560.
Fixes #38562.
Fixes #38563.
Fixes #38569.
Fixes #38578.
Fixes #38600.

_A lot of the early issues fixed by this had previous failures, but nothing very recent or actionable. I think it's worth closing them now that they should be fixed in the short term._

This fixes a bug introduced in 1ff3556 where Raft proposal quota is no longer released when `Replica.propose` fails. This used to happen [here](1ff3556#diff-4315c7ebf8b8bf7bda469e1e7be82690L316), but that code was accidentally lost in the rewrite.

I tracked this down by running a series of `import/tpch/nodes=4` and `scrub/all-checks/tpcc/w=100` roachtests. About half the time, the import would stall after a few hours and the roachtest health reports would start logging lines like: `n1/s1  2.00  metrics  requests.slow.latch`. I tracked the stalled latch acquisition to a stalled proposal quota acquisition by a conflicting command. The range debug page showed the following:

![Screenshot_2019-07-01 r56 Range Debug Cockroach Console](https://user-images.githubusercontent.com/5438456/60554197-8519c780-9d04-11e9-8cf5-6c46ffbcf820.png)

We see that the Leaseholder of the Range has no pending commands but also no available proposal quota. This indicates a proposal quota leak, which led to me finding the lost release in this error case.

The (now confirmed) theory for what went wrong in these roachtests is that they are performing imports, which generate a large number of AddSSTRequests. These requests are typically larger than the available proposal quota for a range, meaning that they request all of its available quota. The effect of this is that if even a single byte of quota is leaked, the entire range will seize up and stall when an AddSSTRequests is issued. Instrumentation revealed that a ChangeReplicas request with a quota size equal to the leaked amount was failing due to the error:
```
received invalid ChangeReplicasTrigger REMOVE_REPLICA((n3,s3):3): updated=[(n1,s1):1 (n4,s4):2 (n2,s2):4] next=5 to remove self (leaseholder)
```
Because of the missing error handling, this quota was not being released back into the pool, causing future requests to get stuck indefinitely waiting for leaked quota, stalling the entire import.

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
@craig
Copy link
Contributor

craig bot commented Jul 3, 2019

Build succeeded

@craig craig bot merged commit ba3813c into cockroachdb:master Jul 3, 2019
@nvanbenschoten nvanbenschoten deleted the nvanbenschoten/releaseQuota branch July 3, 2019 16:42
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment