You can not select more than 25 topics
Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
442 lines
13 KiB
442 lines
13 KiB
// Copyright 2020 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package tikv
|
|
|
|
import (
|
|
"context"
|
|
"io"
|
|
"sync"
|
|
"sync/atomic"
|
|
"time"
|
|
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/kvproto/pkg/coprocessor"
|
|
"github.com/pingcap/kvproto/pkg/kvrpcpb"
|
|
"github.com/pingcap/kvproto/pkg/metapb"
|
|
"github.com/pingcap/tidb/kv"
|
|
"github.com/pingcap/tidb/store/tikv/tikvrpc"
|
|
"github.com/pingcap/tidb/util/logutil"
|
|
"github.com/pingcap/tidb/util/memory"
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
// batchCopTask comprises of multiple copTask that will send to same store.
|
|
type batchCopTask struct {
|
|
storeAddr string
|
|
cmdType tikvrpc.CmdType
|
|
|
|
copTasks []copTaskAndRPCContext
|
|
}
|
|
|
|
type batchCopResponse struct {
|
|
pbResp *coprocessor.BatchResponse
|
|
detail *CopRuntimeStats
|
|
|
|
// batch Cop Response is yet to return startKey. So batchCop cannot retry partially.
|
|
startKey kv.Key
|
|
err error
|
|
respSize int64
|
|
respTime time.Duration
|
|
}
|
|
|
|
// GetData implements the kv.ResultSubset GetData interface.
|
|
func (rs *batchCopResponse) GetData() []byte {
|
|
return rs.pbResp.Data
|
|
}
|
|
|
|
// GetStartKey implements the kv.ResultSubset GetStartKey interface.
|
|
func (rs *batchCopResponse) GetStartKey() kv.Key {
|
|
return rs.startKey
|
|
}
|
|
|
|
// GetExecDetails is unavailable currently, because TiFlash has not collected exec details for batch cop.
|
|
// TODO: Will fix in near future.
|
|
func (rs *batchCopResponse) GetCopRuntimeStats() *CopRuntimeStats {
|
|
return rs.detail
|
|
}
|
|
|
|
// MemSize returns how many bytes of memory this response use
|
|
func (rs *batchCopResponse) MemSize() int64 {
|
|
if rs.respSize != 0 {
|
|
return rs.respSize
|
|
}
|
|
|
|
// ignore rs.err
|
|
rs.respSize += int64(cap(rs.startKey))
|
|
if rs.detail != nil {
|
|
rs.respSize += int64(sizeofExecDetails)
|
|
}
|
|
if rs.pbResp != nil {
|
|
// Using a approximate size since it's hard to get a accurate value.
|
|
rs.respSize += int64(rs.pbResp.Size())
|
|
}
|
|
return rs.respSize
|
|
}
|
|
|
|
func (rs *batchCopResponse) RespTime() time.Duration {
|
|
return rs.respTime
|
|
}
|
|
|
|
type copTaskAndRPCContext struct {
|
|
task *copTask
|
|
ctx *RPCContext
|
|
}
|
|
|
|
func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, req *kv.Request) ([]*batchCopTask, error) {
|
|
start := time.Now()
|
|
const cmdType = tikvrpc.CmdBatchCop
|
|
rangesLen := ranges.len()
|
|
for {
|
|
var tasks []*copTask
|
|
appendTask := func(regionWithRangeInfo *KeyLocation, ranges *copRanges) {
|
|
tasks = append(tasks, &copTask{
|
|
region: regionWithRangeInfo.Region,
|
|
ranges: ranges,
|
|
cmdType: cmdType,
|
|
storeType: req.StoreType,
|
|
})
|
|
}
|
|
|
|
err := splitRanges(bo, cache, ranges, appendTask)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
|
|
var batchTasks []*batchCopTask
|
|
|
|
storeTaskMap := make(map[string]*batchCopTask)
|
|
needRetry := false
|
|
for _, task := range tasks {
|
|
rpcCtx, err := cache.GetTiFlashRPCContext(bo, task.region)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
// If the region is not found in cache, it must be out
|
|
// of date and already be cleaned up. We should retry and generate new tasks.
|
|
if rpcCtx == nil {
|
|
needRetry = true
|
|
logutil.BgLogger().Info("retry for TiFlash peer with region missing", zap.Uint64("region id", task.region.GetID()))
|
|
// Probably all the regions are invalid. Make the loop continue and mark all the regions invalid.
|
|
// Then `splitRegion` will reloads these regions.
|
|
continue
|
|
}
|
|
if batchCop, ok := storeTaskMap[rpcCtx.Addr]; ok {
|
|
batchCop.copTasks = append(batchCop.copTasks, copTaskAndRPCContext{task: task, ctx: rpcCtx})
|
|
} else {
|
|
batchTask := &batchCopTask{
|
|
storeAddr: rpcCtx.Addr,
|
|
cmdType: cmdType,
|
|
copTasks: []copTaskAndRPCContext{{task, rpcCtx}},
|
|
}
|
|
storeTaskMap[rpcCtx.Addr] = batchTask
|
|
}
|
|
}
|
|
if needRetry {
|
|
// Backoff once for each retry.
|
|
err = bo.Backoff(BoRegionMiss, errors.New("Cannot find region with TiFlash peer"))
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
continue
|
|
}
|
|
for _, task := range storeTaskMap {
|
|
batchTasks = append(batchTasks, task)
|
|
}
|
|
|
|
if elapsed := time.Since(start); elapsed > time.Millisecond*500 {
|
|
logutil.BgLogger().Warn("buildBatchCopTasks takes too much time",
|
|
zap.Duration("elapsed", elapsed),
|
|
zap.Int("range len", rangesLen),
|
|
zap.Int("task len", len(batchTasks)))
|
|
}
|
|
tikvTxnRegionsNumHistogramWithBatchCoprocessor.Observe(float64(len(batchTasks)))
|
|
return batchTasks, nil
|
|
}
|
|
}
|
|
|
|
func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response {
|
|
if req.KeepOrder || req.Desc {
|
|
return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")}
|
|
}
|
|
ctx = context.WithValue(ctx, txnStartKey, req.StartTs)
|
|
bo := NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars)
|
|
tasks, err := buildBatchCopTasks(bo, c.store.regionCache, &copRanges{mid: req.KeyRanges}, req)
|
|
if err != nil {
|
|
return copErrorResponse{err}
|
|
}
|
|
it := &batchCopIterator{
|
|
store: c.store,
|
|
req: req,
|
|
finishCh: make(chan struct{}),
|
|
vars: vars,
|
|
memTracker: req.MemTracker,
|
|
clientHelper: clientHelper{
|
|
LockResolver: c.store.lockResolver,
|
|
RegionCache: c.store.regionCache,
|
|
Client: c.store.client,
|
|
minCommitTSPushed: &minCommitTSPushed{data: make(map[uint64]struct{}, 5)},
|
|
},
|
|
rpcCancel: NewRPCanceller(),
|
|
}
|
|
ctx = context.WithValue(ctx, RPCCancellerCtxKey{}, it.rpcCancel)
|
|
it.tasks = tasks
|
|
it.respChan = make(chan *batchCopResponse, 2048)
|
|
go it.run(ctx)
|
|
return it
|
|
}
|
|
|
|
type batchCopIterator struct {
|
|
clientHelper
|
|
|
|
store *tikvStore
|
|
req *kv.Request
|
|
finishCh chan struct{}
|
|
|
|
tasks []*batchCopTask
|
|
|
|
// Batch results are stored in respChan.
|
|
respChan chan *batchCopResponse
|
|
|
|
vars *kv.Variables
|
|
|
|
memTracker *memory.Tracker
|
|
|
|
replicaReadSeed uint32
|
|
|
|
rpcCancel *RPCCanceller
|
|
|
|
wg sync.WaitGroup
|
|
// closed represents when the Close is called.
|
|
// There are two cases we need to close the `finishCh` channel, one is when context is done, the other one is
|
|
// when the Close is called. we use atomic.CompareAndSwap `closed` to to make sure the channel is not closed twice.
|
|
closed uint32
|
|
}
|
|
|
|
func (b *batchCopIterator) run(ctx context.Context) {
|
|
// We run workers for every batch cop.
|
|
for _, task := range b.tasks {
|
|
b.wg.Add(1)
|
|
bo := NewBackofferWithVars(ctx, copNextMaxBackoff, b.vars)
|
|
go b.handleTask(ctx, bo, task)
|
|
}
|
|
b.wg.Wait()
|
|
close(b.respChan)
|
|
}
|
|
|
|
// Next returns next coprocessor result.
|
|
// NOTE: Use nil to indicate finish, so if the returned ResultSubset is not nil, reader should continue to call Next().
|
|
func (b *batchCopIterator) Next(ctx context.Context) (kv.ResultSubset, error) {
|
|
var (
|
|
resp *batchCopResponse
|
|
ok bool
|
|
closed bool
|
|
)
|
|
|
|
// Get next fetched resp from chan
|
|
resp, ok, closed = b.recvFromRespCh(ctx)
|
|
if !ok || closed {
|
|
return nil, nil
|
|
}
|
|
|
|
if resp.err != nil {
|
|
return nil, errors.Trace(resp.err)
|
|
}
|
|
|
|
err := b.store.CheckVisibility(b.req.StartTs)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
return resp, nil
|
|
}
|
|
|
|
func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopResponse, ok bool, exit bool) {
|
|
ticker := time.NewTicker(3 * time.Second)
|
|
defer ticker.Stop()
|
|
for {
|
|
select {
|
|
case resp, ok = <-b.respChan:
|
|
return
|
|
case <-ticker.C:
|
|
if atomic.LoadUint32(b.vars.Killed) == 1 {
|
|
resp = &batchCopResponse{err: ErrQueryInterrupted}
|
|
ok = true
|
|
return
|
|
}
|
|
case <-b.finishCh:
|
|
exit = true
|
|
return
|
|
case <-ctx.Done():
|
|
// We select the ctx.Done() in the thread of `Next` instead of in the worker to avoid the cost of `WithCancel`.
|
|
if atomic.CompareAndSwapUint32(&b.closed, 0, 1) {
|
|
close(b.finishCh)
|
|
}
|
|
exit = true
|
|
return
|
|
}
|
|
}
|
|
}
|
|
|
|
// Close releases the resource.
|
|
func (b *batchCopIterator) Close() error {
|
|
if atomic.CompareAndSwapUint32(&b.closed, 0, 1) {
|
|
close(b.finishCh)
|
|
}
|
|
b.rpcCancel.CancelAll()
|
|
b.wg.Wait()
|
|
return nil
|
|
}
|
|
|
|
func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task *batchCopTask) {
|
|
logutil.BgLogger().Debug("handle batch task")
|
|
tasks := []*batchCopTask{task}
|
|
for idx := 0; idx < len(tasks); idx++ {
|
|
ret, err := b.handleTaskOnce(ctx, bo, tasks[idx])
|
|
if err != nil {
|
|
resp := &batchCopResponse{err: errors.Trace(err), detail: new(CopRuntimeStats)}
|
|
b.sendToRespCh(resp)
|
|
break
|
|
}
|
|
tasks = append(tasks, ret...)
|
|
}
|
|
b.wg.Done()
|
|
}
|
|
|
|
// Merge all ranges and request again.
|
|
func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) {
|
|
ranges := &copRanges{}
|
|
for _, taskCtx := range batchTask.copTasks {
|
|
taskCtx.task.ranges.do(func(ran *kv.KeyRange) {
|
|
ranges.mid = append(ranges.mid, *ran)
|
|
})
|
|
}
|
|
return buildBatchCopTasks(bo, b.RegionCache, ranges, b.req)
|
|
}
|
|
|
|
func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, task *batchCopTask) ([]*batchCopTask, error) {
|
|
logutil.BgLogger().Debug("handle batch task once")
|
|
sender := NewRegionBatchRequestSender(b.store.regionCache, b.store.client)
|
|
var regionInfos []*coprocessor.RegionInfo
|
|
for _, task := range task.copTasks {
|
|
regionInfos = append(regionInfos, &coprocessor.RegionInfo{
|
|
RegionId: task.task.region.id,
|
|
RegionEpoch: &metapb.RegionEpoch{
|
|
ConfVer: task.task.region.confVer,
|
|
Version: task.task.region.ver,
|
|
},
|
|
Ranges: task.task.ranges.toPBRanges(),
|
|
})
|
|
}
|
|
|
|
copReq := coprocessor.BatchRequest{
|
|
Tp: b.req.Tp,
|
|
StartTs: b.req.StartTs,
|
|
Data: b.req.Data,
|
|
SchemaVer: b.req.SchemaVar,
|
|
Regions: regionInfos,
|
|
}
|
|
|
|
req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{
|
|
IsolationLevel: pbIsolationLevel(b.req.IsolationLevel),
|
|
Priority: kvPriorityToCommandPri(b.req.Priority),
|
|
NotFillCache: b.req.NotFillCache,
|
|
HandleTime: true,
|
|
ScanDetail: true,
|
|
TaskId: b.req.TaskID,
|
|
})
|
|
req.StoreTp = kv.TiFlash
|
|
|
|
logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.copTasks)))
|
|
resp, retry, cancel, err := sender.sendStreamReqToAddr(bo, task.copTasks, req, ReadTimeoutUltraLong)
|
|
// If there are store errors, we should retry for all regions.
|
|
if retry {
|
|
return b.retryBatchCopTask(ctx, bo, task)
|
|
}
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
defer cancel()
|
|
return nil, b.handleStreamedBatchCopResponse(ctx, bo, resp.Resp.(*tikvrpc.BatchCopStreamResponse), task)
|
|
}
|
|
|
|
func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) {
|
|
defer response.Close()
|
|
resp := response.BatchResponse
|
|
if resp == nil {
|
|
// streaming request returns io.EOF, so the first Response is nil.
|
|
return
|
|
}
|
|
for {
|
|
err = b.handleBatchCopResponse(bo, resp, task)
|
|
if err != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
resp, err = response.Recv()
|
|
if err != nil {
|
|
if errors.Cause(err) == io.EOF {
|
|
return nil
|
|
}
|
|
|
|
if err1 := bo.Backoff(boTiKVRPC, errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
|
|
// No coprocessor.Response for network error, rebuild task based on the last success one.
|
|
if errors.Cause(err) == context.Canceled {
|
|
logutil.BgLogger().Info("stream recv timeout", zap.Error(err))
|
|
} else {
|
|
logutil.BgLogger().Info("stream unknown error", zap.Error(err))
|
|
}
|
|
return errors.Trace(err)
|
|
}
|
|
}
|
|
}
|
|
|
|
func (b *batchCopIterator) handleBatchCopResponse(bo *Backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) {
|
|
if otherErr := response.GetOtherError(); otherErr != "" {
|
|
err = errors.Errorf("other error: %s", otherErr)
|
|
logutil.BgLogger().Warn("other error",
|
|
zap.Uint64("txnStartTS", b.req.StartTs),
|
|
zap.String("storeAddr", task.storeAddr),
|
|
zap.Error(err))
|
|
return errors.Trace(err)
|
|
}
|
|
|
|
resp := batchCopResponse{
|
|
pbResp: response,
|
|
detail: new(CopRuntimeStats),
|
|
}
|
|
|
|
resp.detail.BackoffTime = time.Duration(bo.totalSleep) * time.Millisecond
|
|
resp.detail.BackoffSleep = make(map[string]time.Duration, len(bo.backoffTimes))
|
|
resp.detail.BackoffTimes = make(map[string]int, len(bo.backoffTimes))
|
|
for backoff := range bo.backoffTimes {
|
|
backoffName := backoff.String()
|
|
resp.detail.BackoffTimes[backoffName] = bo.backoffTimes[backoff]
|
|
resp.detail.BackoffSleep[backoffName] = time.Duration(bo.backoffSleepMS[backoff]) * time.Millisecond
|
|
}
|
|
resp.detail.CalleeAddress = task.storeAddr
|
|
|
|
b.sendToRespCh(&resp)
|
|
|
|
return
|
|
}
|
|
|
|
func (b *batchCopIterator) sendToRespCh(resp *batchCopResponse) (exit bool) {
|
|
select {
|
|
case b.respChan <- resp:
|
|
case <-b.finishCh:
|
|
exit = true
|
|
}
|
|
return
|
|
}
|
|
|