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

executor: add concurrency limit on the union executor #16815

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
55 changes: 52 additions & 3 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1461,19 +1461,66 @@ func (e *UnionExec) Open(ctx context.Context) error {
return nil
}
Copy link
Contributor

@lysu lysu Sep 1, 2020

Choose a reason for hiding this comment

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

@tiancaiamao maybe also need to make child.Open under flow-control...

some executors will buildCopRequest(make region cache or pd became busy) in "Open phase" 😭

github.com/pingcap/tidb/store/tikv.splitRanges(0xc003586178, 0xc00024bf80, 0xc0bb230bd0, 0xc003585f78, 0x100000000000000, 0xc4a1d6)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/store/tikv/coprocessor.go:295 +0x592
github.com/pingcap/tidb/store/tikv.buildCopTasks(0xc003586178, 0xc00024bf80, 0xc0bb230bd0, 0x23b0000, 0x1e70e80, 0xc038b15840, 0x240b760, 0xc0bb230ba0, 0x208
0)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/store/tikv/coprocessor.go:275 +0x114
github.com/pingcap/tidb/store/tikv.(*CopClient).Send(0xc0567eb1a0, 0x240b760, 0xc12e277cb0, 0xc0bba52a80, 0xc02d5a0b00, 0x10, 0x10)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/store/tikv/coprocessor.go:91 +0x205
github.com/pingcap/tidb/distsql.Select(0x240b760, 0xc12e277cb0, 0x2443e00, 0xc03eb2aa50, 0xc0bba52a80, 0xc01ba72100, 0x1f, 0x1f, 0xc01ba0f680, 0xc0035862f0, ...)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/distsql/distsql.go:44 +0x151
github.com/pingcap/tidb/distsql.SelectWithRuntimeStats(0x240b760, 0xc12e277cb0, 0x2443e00, 0xc03eb2aa50, 0xc0bba52a80, 0xc01ba72100, 0x1f, 0x1f, 0xc01ba0f680, 0xc05246ae00, ...)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/distsql/distsql.go:89 +0x97
github.com/pingcap/tidb/executor.selectResultHook.SelectResult(0x0, 0x240b760, 0xc12e277cb0, 0x2443e00, 0xc03eb2aa50, 0xc0bba52a80, 0xc01ba72100, 0x1f, 0x1f, 0xc01ba0f680, ...)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/executor/table_reader.go:48 +0x1c4
github.com/pingcap/tidb/executor.(*TableReaderExecutor).buildResp(0xc0283e6480, 0x240b760, 0xc12e277cb0, 0xc03d9ef018, 0x1, 0x1, 0x1, 0x0, 0x0, 0x0)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/executor/table_reader.go:187 +0x35d
github.com/pingcap/tidb/executor.(*TableReaderExecutor).Open(0xc0283e6480, 0x240b760, 0xc12e277cb0, 0x1f, 0x1f)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/executor/table_reader.go:120 +0x3cd
github.com/pingcap/tidb/executor.(*baseExecutor).Open(0xc01ba703c0, 0x240b760, 0xc12e277cb0, 0x0, 0x0)
        /home/jenkins/agent/workspace/tidb_ghpr_build/go/src/github.com/pingcap/tidb/executor/executor.go:99 +0x7a
github.com/pingcap/tidb/executor.(*LimitExec).Open(0xc01ba703c0, 0x240b760, 0xc12e277cb0, 0x0, 0x0)


var sleepDur = []time.Duration{
time.Microsecond,
50 * time.Microsecond,
100 * time.Microsecond,
500 * time.Microsecond,
time.Millisecond,
2 * time.Millisecond,
3 * time.Millisecond,
4 * time.Millisecond,
5 * time.Millisecond,
6 * time.Millisecond,
}

// generateToken generates tokens to control:
// 1. How many workers in total can be spawn.
// 2. How long to wait before spawning a new worker.
func generateToken(tokenCh chan<- struct{}, exitCh <-chan struct{}) {
t := time.NewTimer(sleepDur[0])
defer t.Stop()
// Generate 10 tokens at most, so the worker count is limited.
for _, duration := range sleepDur {
tokenCh <- struct{}{}
select {
case <-t.C:
t.Reset(duration)
case <-exitCh:
return
}
}
}

func (e *UnionExec) initialize(ctx context.Context) {
e.resultPool = make(chan *unionWorkerResult, len(e.children))
e.resourcePools = make([]chan *chunk.Chunk, len(e.children))

tokenCh := make(chan struct{}, len(sleepDur))
go generateToken(tokenCh, e.finished)

for i := range e.children {
e.resourcePools[i] = make(chan *chunk.Chunk, 1)
e.resourcePools[i] <- e.childrenResults[i]
e.wg.Add(1)
go e.resultPuller(ctx, i)
}
e.wg.Add(len(e.children))
go func() {
for i := range e.children {
select {
// If we do not limit the goroutine count here, 'select * from t limit 1000' on a
// very large partitioned table makes TiDB OOM. Because before the limit take
// effect, the background table reader has already fetch too much data.
case <-tokenCh:
go e.resultPuller(ctx, i, tokenCh)
case <-e.finished:
e.wg.Done()
}
}
}()
go e.waitAllFinished()
}

func (e *UnionExec) resultPuller(ctx context.Context, childID int) {
func (e *UnionExec) resultPuller(ctx context.Context, childID int, tokenCh chan struct{}) {
result := &unionWorkerResult{
err: nil,
chk: nil,
Expand All @@ -1490,6 +1537,8 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) {
e.stopFetchData.Store(true)
}
e.wg.Done()
// Release the token so we can spawn a new resultPuller.
tokenCh <- struct{}{}
}()
for {
if e.stopFetchData.Load().(bool) {
Expand Down
12 changes: 12 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1340,6 +1340,18 @@ func (s *testSuiteP2) TestUnion(c *C) {
tk.MustQuery("select count(distinct a), sum(distinct a), avg(distinct a) from (select a from t union all select b from t) tmp;").Check(testkit.Rows("1 1.000 1.0000000"))
}

func (s *testSuiteP2) TestUnionLimit(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists union_limit")
tk.MustExec("create table union_limit (id int) partition by hash(id) partitions 30")
for i := 0; i < 60; i++ {
tk.MustExec(fmt.Sprintf("insert into union_limit values (%d)", i))
}
// Cover the code for worker count limit in the union executor.
tk.MustQuery("select * from union_limit limit 10")
}

func (s *testSuiteP1) TestNeighbouringProj(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down