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: open doubleread close test and fix bug. #3316

Merged
merged 4 commits into from
May 23, 2017
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
1 change: 0 additions & 1 deletion executor/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import (
// TestIndexDoubleReadClose checks that when a index double read returns before reading all the rows, the goroutine doesn't
// leak. For testing distsql with multiple regions, we need to manually split a mock TiKV.
func (s *testSuite) TestIndexDoubleReadClose(c *C) {
c.Skip("new dist sql use different executor, reopen this test in the future.")
if _, ok := s.store.GetClient().(*tikv.CopClient); !ok {
// Make sure the store is tikv store.
return
Expand Down
31 changes: 18 additions & 13 deletions executor/new_distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -295,6 +295,22 @@ func (e *IndexLookUpExecutor) executeTask(task *lookupTableTask, goCtx goctx.Con
}
}

func (e *IndexLookUpExecutor) pickAndExecTask(workCh <-chan *lookupTableTask) {
txnCtx := e.ctx.GoCtx()
childCtx, cancel := goctx.WithCancel(txnCtx)
defer cancel()
for {
select {
case task := <-workCh:
if task == nil {
return
}
e.executeTask(task, childCtx)
case <-childCtx.Done():
}
Copy link
Contributor

Choose a reason for hiding this comment

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

return here

}
}

// fetchHandlesAndStartWorkers fetches a batch of handles from index data and builds the index lookup tasks.
// We initialize some workers to execute this tasks concurrently and put the task to taskCh by order.
func (e *IndexLookUpExecutor) fetchHandlesAndStartWorkers() {
Expand All @@ -307,22 +323,11 @@ func (e *IndexLookUpExecutor) fetchHandlesAndStartWorkers() {
}()

lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency
txnCtx := e.ctx.GoCtx()
for i := 0; i < lookupConcurrencyLimit; i++ {
go func() {
childCtx, cancel := goctx.WithCancel(txnCtx)
defer cancel()
select {
Copy link
Contributor

Choose a reason for hiding this comment

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

the old code forget the for loop

case task := <-workCh:
if task == nil {
return
}
e.executeTask(task, childCtx)
case <-childCtx.Done():
}
}()
go e.pickAndExecTask(workCh)
}

txnCtx := e.ctx.GoCtx()
for {
handles, finish, err := extractHandlesFromIndexResult(e.result)
if err != nil || finish {
Expand Down