Skip to content

Commit

Permalink
store/copr: revert pr/35975, do not reduce concurrency for keep order…
Browse files Browse the repository at this point in the history
… coprocessor request (#55141)

ref #54969
  • Loading branch information
tiancaiamao authored Aug 2, 2024
1 parent 2945819 commit 1536aa8
Showing 1 changed file with 10 additions and 24 deletions.
34 changes: 10 additions & 24 deletions pkg/store/copr/coprocessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,31 +220,17 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars
}

if it.req.KeepOrder {
// Don't set high concurrency for the keep order case. It wastes a lot of memory and gains nothing.
// TL;DR
// Because for a keep order coprocessor request, the cop tasks are handled one by one, if we set a
// higher concurrency, the data is just cached and not consumed for a while, this increase the memory usage.
// Set concurrency to 2 can reduce the memory usage and I've tested that it does not necessarily
// decrease the performance.
// For ReqTypeAnalyze, we keep its concurrency to avoid slow analyze(see https://github.com/pingcap/tidb/issues/40162 for details).
if it.concurrency > 2 && it.req.Tp != kv.ReqTypeAnalyze {
oldConcurrency := it.concurrency
partitionNum := req.KeyRanges.PartitionNum()
if partitionNum > it.concurrency {
partitionNum = it.concurrency
}
it.concurrency = 2
if it.concurrency < partitionNum {
it.concurrency = partitionNum
}

failpoint.Inject("testRateLimitActionMockConsumeAndAssert", func(val failpoint.Value) {
if val.(bool) {
// When the concurrency is too small, test case tests/realtikvtest/sessiontest.TestCoprocessorOOMAction can't trigger OOM condition
it.concurrency = oldConcurrency
}
})
oldConcurrency := it.concurrency
partitionNum := req.KeyRanges.PartitionNum()
if partitionNum > 0 && partitionNum < it.concurrency {
it.concurrency = partitionNum
}
failpoint.Inject("testRateLimitActionMockConsumeAndAssert", func(val failpoint.Value) {
if val.(bool) {
// When the concurrency is too small, test case tests/realtikvtest/sessiontest.TestCoprocessorOOMAction can't trigger OOM condition
it.concurrency = oldConcurrency
}
})
if it.smallTaskConcurrency > 20 {
it.smallTaskConcurrency = 20
}
Expand Down

0 comments on commit 1536aa8

Please sign in to comment.