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

copr: increase extra concurrency for small coprocessor tasks #37725

Merged
merged 9 commits into from
Oct 14, 2022
Merged
Show file tree
Hide file tree
Changes from 2 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
9 changes: 6 additions & 3 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func (builder *RequestBuilder) SetHandleRangesForTables(sc *stmtctx.StatementCon
// SetTableHandles sets "KeyRanges" for "kv.Request" by converting table handles
// "handles" to "KeyRanges" firstly.
func (builder *RequestBuilder) SetTableHandles(tid int64, handles []kv.Handle) *RequestBuilder {
builder.Request.KeyRanges = TableHandlesToKVRanges(tid, handles)
builder.Request.KeyRanges, builder.FixedRowCountHint = TableHandlesToKVRanges(tid, handles)
return builder
}

Expand Down Expand Up @@ -515,8 +515,9 @@ func SplitRangesAcrossInt64Boundary(ranges []*ranger.Range, keepOrder bool, desc

// TableHandlesToKVRanges converts sorted handle to kv ranges.
// For continuous handles, we should merge them to a single key range.
func TableHandlesToKVRanges(tid int64, handles []kv.Handle) []kv.KeyRange {
func TableHandlesToKVRanges(tid int64, handles []kv.Handle) ([]kv.KeyRange, []int) {
krs := make([]kv.KeyRange, 0, len(handles))
hint := make([]int, 0, len(handles))
i := 0
for i < len(handles) {
if commonHandle, ok := handles[i].(*kv.CommonHandle); ok {
Expand All @@ -525,6 +526,7 @@ func TableHandlesToKVRanges(tid int64, handles []kv.Handle) []kv.KeyRange {
EndKey: tablecodec.EncodeRowKey(tid, kv.Key(commonHandle.Encoded()).Next()),
}
krs = append(krs, ran)
hint = append(hint, 1)
i++
continue
}
Expand All @@ -540,9 +542,10 @@ func TableHandlesToKVRanges(tid int64, handles []kv.Handle) []kv.KeyRange {
startKey := tablecodec.EncodeRowKey(tid, low)
endKey := tablecodec.EncodeRowKey(tid, high)
krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey})
hint = append(hint, j-i)
i = j
}
return krs
return krs, hint
}

// PartitionHandlesToKVRanges convert ParitionHandles to kv ranges.
Expand Down
21 changes: 11 additions & 10 deletions distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ func TestTableHandlesToKVRanges(t *testing.T) {

// Build key ranges.
expect := getExpectedRanges(1, hrs)
actual := TableHandlesToKVRanges(1, handles)
actual, _ := TableHandlesToKVRanges(1, handles)

// Compare key ranges and expected key ranges.
require.Equal(t, len(expect), len(actual))
Expand Down Expand Up @@ -396,15 +396,16 @@ func TestRequestBuilder3(t *testing.T) {
EndKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xf, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x65},
},
},
Cacheable: true,
KeepOrder: false,
Desc: false,
Concurrency: variable.DefDistSQLScanConcurrency,
IsolationLevel: 0,
Priority: 0,
NotFillCache: false,
ReplicaRead: kv.ReplicaReadLeader,
ReadReplicaScope: kv.GlobalReplicaScope,
Cacheable: true,
KeepOrder: false,
Desc: false,
Concurrency: variable.DefDistSQLScanConcurrency,
IsolationLevel: 0,
Priority: 0,
NotFillCache: false,
ReplicaRead: kv.ReplicaReadLeader,
ReadReplicaScope: kv.GlobalReplicaScope,
FixedRowCountHint: []int{1, 4, 2, 1},
}
expect.Paging.MinPagingSize = paging.MinPagingSize
expect.Paging.MaxPagingSize = paging.MaxPagingSize
Expand Down
5 changes: 2 additions & 3 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4148,12 +4148,12 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte
continue
}
handle := kv.IntHandle(content.keys[0].GetInt64())
tmp := distsql.TableHandlesToKVRanges(pid, []kv.Handle{handle})
tmp, _ := distsql.TableHandlesToKVRanges(pid, []kv.Handle{handle})
kvRanges = append(kvRanges, tmp...)
}
} else {
for _, p := range usedPartitionList {
tmp := distsql.TableHandlesToKVRanges(p.GetPhysicalID(), handles)
tmp, _ := distsql.TableHandlesToKVRanges(p.GetPhysicalID(), handles)
kvRanges = append(kvRanges, tmp...)
}
}
Expand Down Expand Up @@ -4287,7 +4287,6 @@ func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Contex
b.SetTableHandles(getPhysicalTableID(e.table), handles)
}
}

return builder.buildTableReaderBase(ctx, e, b)
}

Expand Down
6 changes: 4 additions & 2 deletions executor/mem_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -555,7 +555,8 @@ func (m *memIndexLookUpReader) getMemRows(ctx context.Context) ([][]types.Datum,
continue
}
numHandles += len(handles)
tblKVRanges = append(tblKVRanges, distsql.TableHandlesToKVRanges(getPhysicalTableID(tbl), handles)...)
ranges, _ := distsql.TableHandlesToKVRanges(getPhysicalTableID(tbl), handles)
tblKVRanges = append(tblKVRanges, ranges...)
}
if numHandles == 0 {
return nil, nil
Expand Down Expand Up @@ -687,7 +688,8 @@ func (m *memIndexMergeReader) getMemRows(ctx context.Context) ([][]types.Datum,
continue
}
numHandles += len(handles)
tblKVRanges = append(tblKVRanges, distsql.TableHandlesToKVRanges(getPhysicalTableID(tbl), handles)...)
ranges, _ := distsql.TableHandlesToKVRanges(getPhysicalTableID(tbl), handles)
tblKVRanges = append(tblKVRanges, ranges...)
}

if numHandles == 0 {
Expand Down
2 changes: 2 additions & 0 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -394,6 +394,8 @@ type Request struct {
}
// RequestSource indicates whether the request is an internal request.
RequestSource util.RequestSource
// FixedRowCountHint is the optimization hint for copr request for task scheduling.
FixedRowCountHint []int
}

// CoprRequestAdjuster is used to check and adjust a copr request according to specific rules.
Expand Down
1 change: 0 additions & 1 deletion sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2262,7 +2262,6 @@ type Concurrency struct {
indexLookupJoinConcurrency int

// distSQLScanConcurrency is the number of concurrent dist SQL scan worker.
// distSQLScanConcurrency is deprecated, use ExecutorConcurrency instead.
distSQLScanConcurrency int

// hashJoinConcurrency is the number of concurrent hash join outer worker.
Expand Down
138 changes: 110 additions & 28 deletions store/copr/coprocessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"bytes"
"context"
"fmt"
"math"
"strconv"
"strings"
"sync"
Expand Down Expand Up @@ -66,6 +67,8 @@ var (
const (
copBuildTaskMaxBackoff = 5000
copNextMaxBackoff = 20000
copSmallTaskRow = 6
you06 marked this conversation as resolved.
Show resolved Hide resolved
smallTaskSigma = 0.5
)

// CopClient is coprocessor client.
Expand Down Expand Up @@ -94,11 +97,16 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa
if req.StoreType == kv.TiDB {
// coprocessor on TiDB doesn't support paging
req.Paging.Enable = false
req.FixedRowCountHint = nil
}
if req.Tp != kv.ReqTypeDAG {
// coprocessor request but type is not DAG
req.Paging.Enable = false
}
if req.RequestSource.RequestSourceInternal {
// disable extra concurrency for internal tasks.
req.FixedRowCountHint = nil
}

failpoint.Inject("checkKeyRangeSortedForPaging", func(_ failpoint.Value) {
if req.Paging.Enable {
Expand Down Expand Up @@ -141,6 +149,13 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa
if it.concurrency > len(tasks) {
it.concurrency = len(tasks)
}
if req.FixedRowCountHint != nil {
var smallTasks int
smallTasks, it.smallTaskConcurrency = smallTaskConcurrency(tasks)
if len(tasks)-smallTasks < it.concurrency {
it.concurrency = len(tasks) - smallTasks
}
}
if it.concurrency < 1 {
// Make sure that there is at least one worker.
it.concurrency = 1
Expand All @@ -164,7 +179,10 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa
}
})
}
it.sendRate = util.NewRateLimit(2 * it.concurrency)
if it.smallTaskConcurrency > 20 {
it.smallTaskConcurrency = 20
}
you06 marked this conversation as resolved.
Show resolved Hide resolved
it.sendRate = util.NewRateLimit(2 * (it.concurrency + it.smallTaskConcurrency))
it.respChan = nil
} else {
it.respChan = make(chan *copResponse)
Expand Down Expand Up @@ -198,6 +216,7 @@ type copTask struct {

partitionIndex int64 // used by balanceBatchCopTask in PartitionTableScan
requestSource util.RequestSource
RowCountHint int // used for extra concurrency of small tasks, -1 for unknown row count
}

func (r *copTask) String() string {
Expand Down Expand Up @@ -231,7 +250,8 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv
chanSize = 18
}

var tasks []*copTask
tasks := make([]*copTask, 0, len(locs))
origRangeIdx := 0
for _, loc := range locs {
// TiKV will return gRPC error if the message is too large. So we need to limit the length of the ranges slice
// to make sure the message can be sent successfully.
Expand All @@ -244,6 +264,24 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv
}
for i := 0; i < rLen; {
nextI := mathutil.Min(i+rangesPerTask, rLen)
hint := -1
// calculate the row count hint
if req.FixedRowCountHint != nil {
startKey, endKey := loc.Ranges.At(i).StartKey, loc.Ranges.At(nextI-1).EndKey
// move to the previous range if startKey of current range is lower than endKey of previous location.
you06 marked this conversation as resolved.
Show resolved Hide resolved
if origRangeIdx > 0 && ranges.At(origRangeIdx-1).EndKey.Cmp(startKey) > 0 {
origRangeIdx--
}
hint = 0
for nextOrigRangeIdx := origRangeIdx; nextOrigRangeIdx < ranges.Len(); nextOrigRangeIdx++ {
rangeStart := ranges.At(nextOrigRangeIdx).StartKey
if rangeStart.Cmp(endKey) > 0 {
origRangeIdx = nextOrigRangeIdx
break
}
hint += req.FixedRowCountHint[nextOrigRangeIdx]
}
}
tasks = append(tasks, &copTask{
region: loc.Location.Region,
bucketsVer: loc.getBucketVersion(),
Expand All @@ -255,6 +293,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv
paging: req.Paging.Enable,
pagingSize: pagingSize,
requestSource: req.RequestSource,
RowCountHint: hint,
})
i = nextI
if req.Paging.Enable {
Expand Down Expand Up @@ -290,11 +329,12 @@ func buildTiDBMemCopTasks(ranges *KeyRanges, req *kv.Request) ([]*copTask, error

addr := ser.IP + ":" + strconv.FormatUint(uint64(ser.StatusPort), 10)
tasks = append(tasks, &copTask{
ranges: ranges,
respChan: make(chan *copResponse, 2),
cmdType: cmdType,
storeType: req.StoreType,
storeAddr: addr,
ranges: ranges,
respChan: make(chan *copResponse, 2),
cmdType: cmdType,
storeType: req.StoreType,
storeAddr: addr,
RowCountHint: -1,
})
}
return tasks, nil
Expand All @@ -307,11 +347,37 @@ func reverseTasks(tasks []*copTask) {
}
}

func isSmallTask(task *copTask) bool {
// strictly, only RowCountHint == -1 stands for unknown task rows,
// but when RowCountHint == 0, it may be caused by initialized value,
// to avoid the future bugs, let the tasks with RowCountHint == 0 be non-small tasks.
return task.RowCountHint > 0 && task.RowCountHint <= copSmallTaskRow
}

// smallTaskConcurrency counts the small tasks of tasks,
// then returns the task count and extra concurrency for small tasks.
func smallTaskConcurrency(tasks []*copTask) (int, int) {
res := 0
for _, task := range tasks {
if isSmallTask(task) {
res++
}
}
if res == 0 {
return 0, 0
}
// Calculate the extra concurrency for small tasks
// extra concurrency = tasks / (1 + sigma * sqrt(log(tasks ^ 2)))
extraConc := float64(res) / (1 + smallTaskSigma*math.Sqrt(math.Log(float64(res*res))))
you06 marked this conversation as resolved.
Show resolved Hide resolved
return res, int(extraConc)
}

type copIterator struct {
store *Store
req *kv.Request
concurrency int
finishCh chan struct{}
store *Store
req *kv.Request
concurrency int
smallTaskConcurrency int
finishCh chan struct{}

// If keepOrder, results are stored in copTask.respChan, read them out one by one.
tasks []*copTask
Expand Down Expand Up @@ -366,12 +432,13 @@ type copIteratorWorker struct {

// copIteratorTaskSender sends tasks to taskCh then wait for the workers to exit.
type copIteratorTaskSender struct {
taskCh chan<- *copTask
wg *sync.WaitGroup
tasks []*copTask
finishCh <-chan struct{}
respChan chan<- *copResponse
sendRate *util.RateLimit
taskCh chan<- *copTask
smallTaskCh chan<- *copTask
wg *sync.WaitGroup
tasks []*copTask
finishCh <-chan struct{}
respChan chan<- *copResponse
sendRate *util.RateLimit
}

type copResponse struct {
Expand Down Expand Up @@ -468,11 +535,18 @@ func (worker *copIteratorWorker) run(ctx context.Context) {
// open starts workers and sender goroutines.
func (it *copIterator) open(ctx context.Context, enabledRateLimitAction, enableCollectExecutionInfo bool) {
taskCh := make(chan *copTask, 1)
it.wg.Add(it.concurrency)
smallTaskCh := make(chan *copTask, 1)
it.wg.Add(it.concurrency + it.smallTaskConcurrency)
// Start it.concurrency number of workers to handle cop requests.
for i := 0; i < it.concurrency; i++ {
for i := 0; i < it.concurrency+it.smallTaskConcurrency; i++ {
var ch chan *copTask
if i < it.concurrency {
ch = taskCh
} else {
ch = smallTaskCh
}
worker := &copIteratorWorker{
taskCh: taskCh,
taskCh: ch,
wg: &it.wg,
store: it.store,
req: it.req,
Expand All @@ -488,11 +562,12 @@ func (it *copIterator) open(ctx context.Context, enabledRateLimitAction, enableC
go worker.run(ctx)
}
taskSender := &copIteratorTaskSender{
taskCh: taskCh,
wg: &it.wg,
tasks: it.tasks,
finishCh: it.finishCh,
sendRate: it.sendRate,
taskCh: taskCh,
smallTaskCh: smallTaskCh,
wg: &it.wg,
tasks: it.tasks,
finishCh: it.finishCh,
sendRate: it.sendRate,
}
taskSender.respChan = it.respChan
it.actionOnExceed.setEnabled(enabledRateLimitAction)
Expand All @@ -517,12 +592,19 @@ func (sender *copIteratorTaskSender) run() {
if exit {
break
}
exit = sender.sendToTaskCh(t)
var sendTo chan<- *copTask
if isSmallTask(t) {
sendTo = sender.smallTaskCh
} else {
sendTo = sender.taskCh
}
exit = sender.sendToTaskCh(t, sendTo)
if exit {
break
}
}
close(sender.taskCh)
close(sender.smallTaskCh)

// Wait for worker goroutines to exit.
sender.wg.Wait()
Expand Down Expand Up @@ -569,9 +651,9 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes
}
}

func (sender *copIteratorTaskSender) sendToTaskCh(t *copTask) (exit bool) {
func (sender *copIteratorTaskSender) sendToTaskCh(t *copTask, sendTo chan<- *copTask) (exit bool) {
select {
case sender.taskCh <- t:
case sendTo <- t:
case <-sender.finishCh:
exit = true
}
Expand Down
Loading