-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
kvcoord: setting to reject txns above lock span limit #66927
kvcoord: setting to reject txns above lock span limit #66927
Conversation
Only the last two commits are new here; the rest are #66915. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @andreimatei, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 203 at r5 (raw file):
// Merge and de-dupe in the hope of saving some space. oldLen := len(s.s) + len(spans)
preCondensedLen
maybe?
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 204 at r5 (raw file):
// Merge and de-dupe in the hope of saving some space. oldLen := len(s.s) + len(spans) spans = append(spans, s.s...)
Is this the place where we need to estimate and then recompute size of the result instead of original spanset size?
37072e1
to
7782ccf
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 203 at r5 (raw file):
Previously, aliher1911 (Oleg) wrote…
preCondensedLen
maybe?
did something
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 204 at r5 (raw file):
Previously, aliher1911 (Oleg) wrote…
Is this the place where we need to estimate and then recompute size of the result instead of original spanset size?
yeah... the code around here was all broken; don't know what I've uploaded. Please see now.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 10 of 10 files at r6, 10 of 10 files at r7.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @andreimatei)
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 178 at r7 (raw file):
} // estimateSize returns the size that the spanSet would grow to if spans were
Can we add a unit test around this method to condensable_span_set_test.go
?
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 199 at r7 (raw file):
spans = append(spans, s.s...) lenBeforeMerge := len(spans) spans, _ = roachpb.MergeSpans(&spans)
Will this become quadratic if s.s
is the source of most of the duplication? Should we first roachpb.MergeSpans
s.s
in place (s.mergeAndSort()
), check if we're still exceeding the threshold, and only then merge in the provided spans
?
EDIT: maybe we don't need this because of the call to mergeAndSort in updateLockTracking
. Is that the right place to do this? I'm not sure. I think that still risks this being quadratic because we might never hit that path but keep hitting this path if we overestimate the size of spans on the way in. What do you think?
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 212 at r7 (raw file):
} // Bytes returns the size of the tracked spans.
nit: we're not exporting any other methods from this non-exported type, so let's not export this.
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 274 at r7 (raw file):
// does, we reject it. Note that this check is not precise because generally // we can't know exactly the size of the locks that will be taken by a // request; even if the check passes, we might end up over budget.
Could you give an example of when this is that case? Is this because of resume spans?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 295 at r7 (raw file):
condenseOpt := condenseLocksIfOverBudget if rejectOverBudget {
rejectOverBudget
is already a bool, so this seems a little verbose for not a lot of benefit. I get the use of a typedef in certain cases when a policy needs to be passed through a few layers and it's a result of a complex computation so we're reducing complexity, but that's not the case here, this adds complexity. Why not just pass rejectOverBudget
to updateLockTracking
?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 316 at r7 (raw file):
// tracking budget, an error is returned. // // We can only estimate what what spans this request would end up locking
"what what"
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 330 at r7 (raw file):
// Bail early if the current request is not locking, even if we are already // over budget. In particular, we definitely want to permit rollbacks. We also // want to permit commits, since the damage in taking too much memory has
"lone commits"
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 356 at r7 (raw file):
func (tp *txnPipeliner) markTxnOverflow() { if tp.lockSpanBudgetExceeded {
So lockSpanBudgetExceeded
is just to ensure that we don't double-count TxnsRejectedByLockSpanBudget
? It's not to continue to reject requests after any single request overflowed the budget?
Want to leave that as a comment on lockSpanBudgetExceeded
? Also, rename this method to markLockSpanBudgetExceeded
.
Also, should we somehow distinguish that this field and this method are only related to the lock span budget being exceeded when kv.transaction.reject_over_max_intents_budget.enabled is true?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1101 at r7 (raw file):
details = &errorspb.StringsPayload{ Details: []string{ strconv.Itoa(int(t.lockSpansBytes)), strconv.Itoa(int(t.limitBytes)), t.baSummary, t.txnDetails,
nit: Want to use FormatInt
and ParseInt
so you don't have the int->int64 and int64->int casts.
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1137 at r7 (raw file):
func init() { pKey := errors.GetTypeKey(lockSpansOverBudgetError{}) errors.RegisterLeafEncoder(pKey, encodeLockSpansOverBudgetError)
Out of curiosity, why do we need this? This error will never make its way over a network, will it?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1698 at r7 (raw file):
// inhibits the asyncConsensus flag. func putBatchNoAsyncConsensus(key roachpb.Key, value []byte) roachpb.BatchRequest { ba := roachpb.BatchRequest{}
nit: want to use putBatch
to avoid duplication?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1754 at r7 (raw file):
largeAs[i] = 'a' } largeWrite := putBatch(largeAs, nil)
Are these providing any value? Most of the test cases just create requests inline.
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1859 at r7 (raw file):
_, pErr := tp.SendLocked(ctx, ba) if i == tc.expRejectIdx { if pErr == nil {
require.NotNil
here?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1865 at r7 (raw file):
budgetErr := (lockSpansOverBudgetError{}) if !errors.As(pErr.GoError(), &budgetErr) { t.Fatalf("expected lockSpansOverBudgetError, got %+v", pErr.GoError())
Did you want require.IsType
?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1870 at r7 (raw file):
require.Equal(t, int64(1), tp.txnMetrics.TxnsRejectedByLockSpanBudget.Count()) // Make sure rolling back the txn works.
Should we also demonstrate that rolling forward (committing) works?
pkg/testutils/lint/lint_test.go, line 1149 at r7 (raw file):
":!*.pb.go", ":!*.pb.gw.go", ":!kv/kvclient/kvcoord/txn_interceptor_pipeliner.go",
What's this for?
The limit in question is expressed as a "maximum", which generally implies it's inclusive. Release note: None
7782ccf
to
47abbb8
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 178 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Can we add a unit test around this method to
condensable_span_set_test.go
?
done
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 199 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Will this become quadratic if
s.s
is the source of most of the duplication? Should we firstroachpb.MergeSpans
s.s
in place (s.mergeAndSort()
), check if we're still exceeding the threshold, and only then merge in the providedspans
?EDIT: maybe we don't need this because of the call to mergeAndSort in
updateLockTracking
. Is that the right place to do this? I'm not sure. I think that still risks this being quadratic because we might never hit that path but keep hitting this path if we overestimate the size of spans on the way in. What do you think?
I was thinking that s
is generally already fairly de-duplicated because, if it was closed to the budget, the updateLockTracking
call would have indeed kicked in.
But you're generally right - I think it's a good idea to sort s
in place here if it looks like we're cutting it close. Please see now.
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 212 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: we're not exporting any other methods from this non-exported type, so let's not export this.
renamed to bytesSize()
to not clash with the field
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 274 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Could you give an example of when this is that case? Is this because of resume spans?
Yeah. I added a "think ResumeSpan"
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 295 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
rejectOverBudget
is already a bool, so this seems a little verbose for not a lot of benefit. I get the use of a typedef in certain cases when a policy needs to be passed through a few layers and it's a result of a complex computation so we're reducing complexity, but that's not the case here, this adds complexity. Why not just passrejectOverBudget
toupdateLockTracking
?
done - except I pass in the negation
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 316 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
"what what"
what
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 330 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
"lone commits"
done
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 356 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
So
lockSpanBudgetExceeded
is just to ensure that we don't double-countTxnsRejectedByLockSpanBudget
? It's not to continue to reject requests after any single request overflowed the budget?Want to leave that as a comment on
lockSpanBudgetExceeded
? Also, rename this method tomarkLockSpanBudgetExceeded
.Also, should we somehow distinguish that this field and this method are only related to the lock span budget being exceeded when kv.transaction.reject_over_max_intents_budget.enabled is true?
I think this method and the field were just vestigial from a different attempt at this change; I've removed them. Since we're returning an error when the budget is about to be exceeded, we only expect a rollback afterwards, so the counter should not be incremented repeatedly.
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1101 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: Want to use
FormatInt
andParseInt
so you don't have the int->int64 and int64->int casts.
done
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1111 at r7 (raw file):
_ context.Context, msgPrefix string, safeDetails []string, payload proto.Message, ) error { m, ok := payload.(*errorspb.StringsPayload)
Hey @knz the linter says invalid direct cast on error object
. But I think the direct cast makes sense here. So I'm adding an exception here like others before me. But, instead of these horrible greps, do you think we could add a pragma that the linter recognizes? I've looked at the linter a bit - it traverses the AST, so I dunno what it can look at and what it can't. But maybe you do?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1137 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Out of curiosity, why do we need this? This error will never make its way over a network, will it?
it doesn't make it over the network, but the way in which it's stored into a pErr
forced me to implement these guys nevertheless. I think the pErr
always encodes the inner error as a proto, for some reason.
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1698 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
nit: want to use
putBatch
to avoid duplication?
done
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1754 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Are these providing any value? Most of the test cases just create requests inline.
I think mediumWrite
provides some value through its name. And then I think it largeAs
makes sense to exist as well.
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1859 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
require.NotNil
here?
done
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1865 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Did you want
require.IsType
?
I don't think so, because the type of pErr.GoErr()
is *pgerror.withCandidateCode
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go, line 1870 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should we also demonstrate that rolling forward (committing) works?
done as a new testcase
pkg/testutils/lint/lint_test.go, line 1149 at r7 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
What's this for?
because of this function:
func encodeLockSpansOverBudgetError(
_ context.Context, err error,
) (msgPrefix string, safe []string, details proto.Message) {
which uses proto.Message instead of protoutil.Message, because it's forced to do so by the errors lib
47abbb8
to
fe57198
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
although I think we should try to resolve some of your questions about the linter changes. It seems unfortunate to need to add exceptions to the linter whenever we use this leaf encoder framework.
Reviewed 3 of 9 files at r8, 11 of 11 files at r9.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @andreimatei)
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 203 at r9 (raw file):
// operating on a copy avoids the risk of quadratic behavior over a series of // estimateSize() calls, where each call has to repeatedly merge a copy in // order to discover that the merger saves a lot of space.
s/the merges saves a lot of space/the merge saves enough space to stay under the threshold/
pkg/kv/kvclient/kvcoord/condensable_span_set_test.go, line 71 at r9 (raw file):
mergeThreshold: 5, expEstimate: 2, },
What about a case where the new spans do not fit?
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1137 at r7 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
it doesn't make it over the network, but the way in which it's stored into a
pErr
forced me to implement these guys nevertheless. I think thepErr
always encodes the inner error as a proto, for some reason.
👍
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 635 at r9 (raw file):
} } else { tp.lockFootprint.mergeAndSort()
Now that we're doing this reliably in estimateSize
, is there a benefit to doing it here too?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @andreimatei)
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1111 at r7 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Hey @knz the linter says
invalid direct cast on error object
. But I think the direct cast makes sense here. So I'm adding an exception here like others before me. But, instead of these horrible greps, do you think we could add a pragma that the linter recognizes? I've looked at the linter a bit - it traverses the AST, so I dunno what it can look at and what it can't. But maybe you do?
-
for the short term, I'm in favor of an exception, but please create as separate source file with just this logic so that the exception doesn't mistakenly apply to the rest of the code.
-
yes, improving the linter is better. I suppose we can do what you say, given that's how we handle the other comment-based exceptions. Can you file an issue? Then cc Andrew and me on it.
fe57198
to
684c5c3
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, @knz, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/condensable_span_set.go, line 203 at r9 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
s/the merges saves a lot of space/the merge saves enough space to stay under the threshold/
done
pkg/kv/kvclient/kvcoord/condensable_span_set_test.go, line 71 at r9 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
What about a case where the new spans do not fit?
done
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 1111 at r7 (raw file):
Previously, knz (kena) wrote…
for the short term, I'm in favor of an exception, but please create as separate source file with just this logic so that the exception doesn't mistakenly apply to the rest of the code.
yes, improving the linter is better. I suppose we can do what you say, given that's how we handle the other comment-based exceptions. Can you file an issue? Then cc Andrew and me on it.
moved to different file and files #67860
pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 635 at r9 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Now that we're doing this reliably in
estimateSize
, is there a benefit to doing it here too?
there isn't. removed.
684c5c3
to
4706883
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 5 of 5 files at r10, 1 of 1 files at r11.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @aayushshah15, @aliher1911, and @knz)
This patch introduces kv.transaction.reject_over_max_intents_budget. If set, this changes our behavior when a txn exceeds its locks+in-flight write budget (kv.transaction.max_intents_bytes): instead of compacting some of its lock spans with precision loss, the request causing the budget to be exceeded will be rejected instead. The idea is that we've seen transactions that exceed this budget be very expensive to clean up - they have to scan a lot to find their intents, and these cleanups take wide latches. So now one has the option to reject these transactions, instead of risking this performance cliff. Each request is checked against the budget by the pipeliner before being sent out for evaluation. This check is not precise, since the exact effects of the request on the memory budget are only known at response time because of ResumeSpans, effects of QueryIntents, etc. So, the check is best-effort. If a slips through and then the response overflows the budget, we keep the locks non-condensed; if a further request in the txn tries to lock more, it'll be rejected. A commit/rollback is always allowed to pass through, since it doesn't lock anything by itself. Fixes cockroachdb#66742 Release note (general change): A new cluster setting (kv.transaction.reject_over_max_intents_budget) affords control over the behavior when a transaction exceeds its "locks-tracking memory budget" (dictated by kv.transaction.max_intents_bytes). Instead of allowing such transaction to continue with imprecise tracking of their locks, setting this new option rejects the query that would push its transaction over this budget with an error (error code 53400 - "configuration limit exceeded). Transactions that don't track their locks precisely are potentially destabilizing for the cluster since cleaning them up can take considerable resources. Transactions that change many rows have the potential to run into this memory budget issue.
4706883
to
66903db
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
bors r+
Reviewable status: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @aayushshah15, @aliher1911, @knz, and @nvanbenschoten)
Build failed (retrying...): |
Build succeeded: |
Is a backport on the table here? |
This patch introduces kv.transaction.reject_over_max_intents_budget. If
set, this changes our behavior when a txn exceeds its locks+in-flight
write budget (kv.transaction.max_intents_bytes): instead of compacting
some of its lock spans with precision loss, the request causing the
budget to be exceeded will be rejected instead.
The idea is that we've seen transactions that exceed this budget be very
expensive to clean up - they have to scan a lot to find their intents,
and these cleanups take wide latches. So now one has the option to
reject these transactions, instead of risking this performance cliff.
Each request is checked against the budget by the pipeliner before being
sent out for evaluation. This check is not precise, since the exact
effects of the request on the memory budget are only known at response
time because of ResumeSpans, effects of QueryIntents, etc. So, the check
is best-effort. If a slips through and then the response overflows the
budget, we keep the locks non-condensed; if a further request in the txn
tries to lock more, it'll be rejected. A commit/rollback is
always allowed to pass through, since it doesn't lock anything by
itself.
Fixes #66742
Release note (general change): A new cluster setting
(kv.transaction.reject_over_max_intents_budget) affords control over the
behavior when a transaction exceeds its "locks-tracking memory budget"
(dictated by kv.transaction.max_intents_bytes). Instead of allowing such
transaction to continue with imprecise tracking of their locks, setting
this new option rejects the query that would push its transaction over
this budget with an error (error code 53400 - "configuration limit
exceeded). Transactions that don't track their locks precisely are
potentially destabilizing for the cluster since cleaning them up can
take considerable resources. Transactions that change many rows have the
potential to run into this memory budget issue.