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.
469 lines
14 KiB
469 lines
14 KiB
// Copyright 2018 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 distsql
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"fmt"
|
|
"sort"
|
|
"strconv"
|
|
"sync/atomic"
|
|
"time"
|
|
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/parser/terror"
|
|
"github.com/pingcap/tidb/errno"
|
|
"github.com/pingcap/tidb/kv"
|
|
"github.com/pingcap/tidb/metrics"
|
|
"github.com/pingcap/tidb/sessionctx"
|
|
"github.com/pingcap/tidb/statistics"
|
|
"github.com/pingcap/tidb/store/tikv"
|
|
"github.com/pingcap/tidb/store/tikv/tikvrpc"
|
|
"github.com/pingcap/tidb/types"
|
|
"github.com/pingcap/tidb/util/chunk"
|
|
"github.com/pingcap/tidb/util/codec"
|
|
"github.com/pingcap/tidb/util/dbterror"
|
|
"github.com/pingcap/tidb/util/execdetails"
|
|
"github.com/pingcap/tidb/util/logutil"
|
|
"github.com/pingcap/tidb/util/memory"
|
|
"github.com/pingcap/tipb/go-tipb"
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
var (
|
|
errQueryInterrupted = dbterror.ClassExecutor.NewStd(errno.ErrQueryInterrupted)
|
|
)
|
|
|
|
var (
|
|
coprCacheHistogramHit = metrics.DistSQLCoprCacheHistogram.WithLabelValues("hit")
|
|
coprCacheHistogramMiss = metrics.DistSQLCoprCacheHistogram.WithLabelValues("miss")
|
|
)
|
|
|
|
var (
|
|
_ SelectResult = (*selectResult)(nil)
|
|
_ SelectResult = (*streamResult)(nil)
|
|
)
|
|
|
|
// SelectResult is an iterator of coprocessor partial results.
|
|
type SelectResult interface {
|
|
// Fetch fetches partial results from client.
|
|
Fetch(context.Context)
|
|
// NextRaw gets the next raw result.
|
|
NextRaw(context.Context) ([]byte, error)
|
|
// Next reads the data into chunk.
|
|
Next(context.Context, *chunk.Chunk) error
|
|
// Close closes the iterator.
|
|
Close() error
|
|
}
|
|
|
|
type selectResult struct {
|
|
label string
|
|
resp kv.Response
|
|
|
|
rowLen int
|
|
fieldTypes []*types.FieldType
|
|
ctx sessionctx.Context
|
|
|
|
selectResp *tipb.SelectResponse
|
|
selectRespSize int // record the selectResp.Size() when it is initialized.
|
|
respChkIdx int
|
|
respChunkDecoder *chunk.Decoder
|
|
|
|
feedback *statistics.QueryFeedback
|
|
partialCount int64 // number of partial results.
|
|
sqlType string
|
|
encodeType tipb.EncodeType
|
|
|
|
// copPlanIDs contains all copTasks' planIDs,
|
|
// which help to collect copTasks' runtime stats.
|
|
copPlanIDs []int
|
|
rootPlanID int
|
|
|
|
fetchDuration time.Duration
|
|
durationReported bool
|
|
memTracker *memory.Tracker
|
|
|
|
stats *selectResultRuntimeStats
|
|
}
|
|
|
|
func (r *selectResult) Fetch(ctx context.Context) {
|
|
}
|
|
|
|
func (r *selectResult) fetchResp(ctx context.Context) error {
|
|
for {
|
|
r.respChkIdx = 0
|
|
startTime := time.Now()
|
|
resultSubset, err := r.resp.Next(ctx)
|
|
duration := time.Since(startTime)
|
|
r.fetchDuration += duration
|
|
if err != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
if r.selectResp != nil {
|
|
r.memConsume(-int64(r.selectRespSize))
|
|
}
|
|
if resultSubset == nil {
|
|
r.selectResp = nil
|
|
if !r.durationReported {
|
|
// final round of fetch
|
|
// TODO: Add a label to distinguish between success or failure.
|
|
// https://github.com/pingcap/tidb/issues/11397
|
|
metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds())
|
|
r.durationReported = true
|
|
}
|
|
return nil
|
|
}
|
|
r.selectResp = new(tipb.SelectResponse)
|
|
err = r.selectResp.Unmarshal(resultSubset.GetData())
|
|
if err != nil {
|
|
return errors.Trace(err)
|
|
}
|
|
r.selectRespSize = r.selectResp.Size()
|
|
r.memConsume(int64(r.selectRespSize))
|
|
if err := r.selectResp.Error; err != nil {
|
|
return dbterror.ClassTiKV.Synthesize(terror.ErrCode(err.Code), err.Msg)
|
|
}
|
|
sessVars := r.ctx.GetSessionVars()
|
|
if atomic.LoadUint32(&sessVars.Killed) == 1 {
|
|
return errors.Trace(errQueryInterrupted)
|
|
}
|
|
sc := sessVars.StmtCtx
|
|
for _, warning := range r.selectResp.Warnings {
|
|
sc.AppendWarning(dbterror.ClassTiKV.Synthesize(terror.ErrCode(warning.Code), warning.Msg))
|
|
}
|
|
r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts)
|
|
r.partialCount++
|
|
|
|
hasStats, ok := resultSubset.(CopRuntimeStats)
|
|
if ok {
|
|
copStats := hasStats.GetCopRuntimeStats()
|
|
if copStats != nil {
|
|
r.updateCopRuntimeStats(ctx, copStats, resultSubset.RespTime())
|
|
copStats.CopTime = duration
|
|
sc.MergeExecDetails(&copStats.ExecDetails, nil)
|
|
}
|
|
}
|
|
if len(r.selectResp.Chunks) != 0 {
|
|
break
|
|
}
|
|
}
|
|
if r.stats != nil {
|
|
coprCacheHistogramHit.Observe(float64(r.stats.CoprCacheHitNum))
|
|
coprCacheHistogramMiss.Observe(float64(len(r.stats.copRespTime) - int(r.stats.CoprCacheHitNum)))
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (r *selectResult) Next(ctx context.Context, chk *chunk.Chunk) error {
|
|
chk.Reset()
|
|
if r.selectResp == nil || r.respChkIdx == len(r.selectResp.Chunks) {
|
|
err := r.fetchResp(ctx)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if r.selectResp == nil {
|
|
return nil
|
|
}
|
|
}
|
|
// TODO(Shenghui Wu): add metrics
|
|
switch r.selectResp.GetEncodeType() {
|
|
case tipb.EncodeType_TypeDefault:
|
|
return r.readFromDefault(ctx, chk)
|
|
case tipb.EncodeType_TypeChunk:
|
|
return r.readFromChunk(ctx, chk)
|
|
}
|
|
return errors.Errorf("unsupported encode type:%v", r.encodeType)
|
|
}
|
|
|
|
// NextRaw returns the next raw partial result.
|
|
func (r *selectResult) NextRaw(ctx context.Context) (data []byte, err error) {
|
|
resultSubset, err := r.resp.Next(ctx)
|
|
r.partialCount++
|
|
r.feedback.Invalidate()
|
|
if resultSubset != nil && err == nil {
|
|
data = resultSubset.GetData()
|
|
}
|
|
return data, err
|
|
}
|
|
|
|
func (r *selectResult) readFromDefault(ctx context.Context, chk *chunk.Chunk) error {
|
|
for !chk.IsFull() {
|
|
if r.respChkIdx == len(r.selectResp.Chunks) {
|
|
err := r.fetchResp(ctx)
|
|
if err != nil || r.selectResp == nil {
|
|
return err
|
|
}
|
|
}
|
|
err := r.readRowsData(chk)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if len(r.selectResp.Chunks[r.respChkIdx].RowsData) == 0 {
|
|
r.respChkIdx++
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (r *selectResult) readFromChunk(ctx context.Context, chk *chunk.Chunk) error {
|
|
if r.respChunkDecoder == nil {
|
|
r.respChunkDecoder = chunk.NewDecoder(
|
|
chunk.NewChunkWithCapacity(r.fieldTypes, 0),
|
|
r.fieldTypes,
|
|
)
|
|
}
|
|
|
|
for !chk.IsFull() {
|
|
if r.respChkIdx == len(r.selectResp.Chunks) {
|
|
err := r.fetchResp(ctx)
|
|
if err != nil || r.selectResp == nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
if r.respChunkDecoder.IsFinished() {
|
|
r.respChunkDecoder.Reset(r.selectResp.Chunks[r.respChkIdx].RowsData)
|
|
}
|
|
// If the next chunk size is greater than required rows * 0.8, reuse the memory of the next chunk and return
|
|
// immediately. Otherwise, splice the data to one chunk and wait the next chunk.
|
|
if r.respChunkDecoder.RemainedRows() > int(float64(chk.RequiredRows())*0.8) {
|
|
if chk.NumRows() > 0 {
|
|
return nil
|
|
}
|
|
r.respChunkDecoder.ReuseIntermChk(chk)
|
|
r.respChkIdx++
|
|
return nil
|
|
}
|
|
r.respChunkDecoder.Decode(chk)
|
|
if r.respChunkDecoder.IsFinished() {
|
|
r.respChkIdx++
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *tikv.CopRuntimeStats, respTime time.Duration) {
|
|
callee := copStats.CalleeAddress
|
|
if r.rootPlanID <= 0 || r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" {
|
|
return
|
|
}
|
|
if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) {
|
|
logutil.Logger(ctx).Error("invalid cop task execution summaries length",
|
|
zap.Int("expected", len(r.copPlanIDs)),
|
|
zap.Int("received", len(r.selectResp.GetExecutionSummaries())))
|
|
|
|
return
|
|
}
|
|
if r.stats == nil {
|
|
id := r.rootPlanID
|
|
r.stats = &selectResultRuntimeStats{
|
|
backoffSleep: make(map[string]time.Duration),
|
|
rpcStat: tikv.NewRegionRequestRuntimeStats(),
|
|
}
|
|
r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(id, r.stats)
|
|
}
|
|
r.stats.mergeCopRuntimeStats(copStats, respTime)
|
|
|
|
for i, detail := range r.selectResp.GetExecutionSummaries() {
|
|
if detail != nil && detail.TimeProcessedNs != nil &&
|
|
detail.NumProducedRows != nil && detail.NumIterations != nil {
|
|
planID := r.copPlanIDs[i]
|
|
r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.
|
|
RecordOneCopTask(planID, callee, detail)
|
|
}
|
|
}
|
|
}
|
|
|
|
func (r *selectResult) readRowsData(chk *chunk.Chunk) (err error) {
|
|
rowsData := r.selectResp.Chunks[r.respChkIdx].RowsData
|
|
decoder := codec.NewDecoder(chk, r.ctx.GetSessionVars().Location())
|
|
for !chk.IsFull() && len(rowsData) > 0 {
|
|
for i := 0; i < r.rowLen; i++ {
|
|
rowsData, err = decoder.DecodeOne(rowsData, i, r.fieldTypes[i])
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
}
|
|
r.selectResp.Chunks[r.respChkIdx].RowsData = rowsData
|
|
return nil
|
|
}
|
|
|
|
func (r *selectResult) memConsume(bytes int64) {
|
|
if r.memTracker != nil {
|
|
r.memTracker.Consume(bytes)
|
|
}
|
|
}
|
|
|
|
// Close closes selectResult.
|
|
func (r *selectResult) Close() error {
|
|
if r.feedback.Actual() >= 0 {
|
|
metrics.DistSQLScanKeysHistogram.Observe(float64(r.feedback.Actual()))
|
|
}
|
|
metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount))
|
|
if r.selectResp != nil {
|
|
r.memConsume(-int64(r.selectRespSize))
|
|
}
|
|
return r.resp.Close()
|
|
}
|
|
|
|
// CopRuntimeStats is a interface uses to check whether the result has cop runtime stats.
|
|
type CopRuntimeStats interface {
|
|
// GetCopRuntimeStats gets the cop runtime stats information.
|
|
GetCopRuntimeStats() *tikv.CopRuntimeStats
|
|
}
|
|
|
|
type selectResultRuntimeStats struct {
|
|
copRespTime []time.Duration
|
|
procKeys []int64
|
|
backoffSleep map[string]time.Duration
|
|
totalProcessTime time.Duration
|
|
totalWaitTime time.Duration
|
|
rpcStat tikv.RegionRequestRuntimeStats
|
|
CoprCacheHitNum int64
|
|
}
|
|
|
|
func (s *selectResultRuntimeStats) mergeCopRuntimeStats(copStats *tikv.CopRuntimeStats, respTime time.Duration) {
|
|
s.copRespTime = append(s.copRespTime, respTime)
|
|
s.procKeys = append(s.procKeys, copStats.ProcessedKeys)
|
|
|
|
for k, v := range copStats.BackoffSleep {
|
|
s.backoffSleep[k] += v
|
|
}
|
|
s.totalProcessTime += copStats.ProcessTime
|
|
s.totalWaitTime += copStats.WaitTime
|
|
s.rpcStat.Merge(copStats.RegionRequestRuntimeStats)
|
|
if copStats.CoprCacheHit {
|
|
s.CoprCacheHitNum++
|
|
}
|
|
}
|
|
|
|
func (s *selectResultRuntimeStats) Clone() execdetails.RuntimeStats {
|
|
newRs := selectResultRuntimeStats{
|
|
copRespTime: make([]time.Duration, 0, len(s.copRespTime)),
|
|
procKeys: make([]int64, 0, len(s.procKeys)),
|
|
backoffSleep: make(map[string]time.Duration, len(s.backoffSleep)),
|
|
rpcStat: tikv.NewRegionRequestRuntimeStats(),
|
|
CoprCacheHitNum: s.CoprCacheHitNum,
|
|
}
|
|
newRs.copRespTime = append(newRs.copRespTime, s.copRespTime...)
|
|
newRs.procKeys = append(newRs.procKeys, s.procKeys...)
|
|
for k, v := range s.backoffSleep {
|
|
newRs.backoffSleep[k] += v
|
|
}
|
|
newRs.totalProcessTime += s.totalProcessTime
|
|
newRs.totalWaitTime += s.totalWaitTime
|
|
for k, v := range s.rpcStat.Stats {
|
|
newRs.rpcStat.Stats[k] = v
|
|
}
|
|
return &newRs
|
|
}
|
|
|
|
func (s *selectResultRuntimeStats) Merge(rs execdetails.RuntimeStats) {
|
|
other, ok := rs.(*selectResultRuntimeStats)
|
|
if !ok {
|
|
return
|
|
}
|
|
s.copRespTime = append(s.copRespTime, other.copRespTime...)
|
|
s.procKeys = append(s.procKeys, other.procKeys...)
|
|
|
|
for k, v := range other.backoffSleep {
|
|
s.backoffSleep[k] += v
|
|
}
|
|
s.totalProcessTime += other.totalProcessTime
|
|
s.totalWaitTime += other.totalWaitTime
|
|
s.rpcStat.Merge(other.rpcStat)
|
|
s.CoprCacheHitNum += other.CoprCacheHitNum
|
|
}
|
|
|
|
func (s *selectResultRuntimeStats) String() string {
|
|
buf := bytes.NewBuffer(nil)
|
|
rpcStat := s.rpcStat
|
|
if len(s.copRespTime) > 0 {
|
|
size := len(s.copRespTime)
|
|
if size == 1 {
|
|
buf.WriteString(fmt.Sprintf("cop_task: {num: 1, max:%v, proc_keys: %v", s.copRespTime[0], s.procKeys[0]))
|
|
} else {
|
|
sort.Slice(s.copRespTime, func(i, j int) bool {
|
|
return s.copRespTime[i] < s.copRespTime[j]
|
|
})
|
|
vMax, vMin := s.copRespTime[size-1], s.copRespTime[0]
|
|
vP95 := s.copRespTime[size*19/20]
|
|
sum := 0.0
|
|
for _, t := range s.copRespTime {
|
|
sum += float64(t)
|
|
}
|
|
vAvg := time.Duration(sum / float64(size))
|
|
|
|
sort.Slice(s.procKeys, func(i, j int) bool {
|
|
return s.procKeys[i] < s.procKeys[j]
|
|
})
|
|
keyMax := s.procKeys[size-1]
|
|
keyP95 := s.procKeys[size*19/20]
|
|
buf.WriteString(fmt.Sprintf("cop_task: {num: %v, max: %v, min: %v, avg: %v, p95: %v", size, vMax, vMin, vAvg, vP95))
|
|
if keyMax > 0 {
|
|
buf.WriteString(", max_proc_keys: ")
|
|
buf.WriteString(strconv.FormatInt(keyMax, 10))
|
|
buf.WriteString(", p95_proc_keys: ")
|
|
buf.WriteString(strconv.FormatInt(keyP95, 10))
|
|
}
|
|
if s.totalProcessTime > 0 {
|
|
buf.WriteString(", tot_proc: ")
|
|
buf.WriteString(s.totalProcessTime.String())
|
|
if s.totalWaitTime > 0 {
|
|
buf.WriteString(", tot_wait: ")
|
|
buf.WriteString(s.totalWaitTime.String())
|
|
}
|
|
}
|
|
}
|
|
copRPC := rpcStat.Stats[tikvrpc.CmdCop]
|
|
if copRPC != nil && copRPC.Count > 0 {
|
|
rpcStat = rpcStat.Clone()
|
|
delete(rpcStat.Stats, tikvrpc.CmdCop)
|
|
buf.WriteString(", rpc_num: ")
|
|
buf.WriteString(strconv.FormatInt(copRPC.Count, 10))
|
|
buf.WriteString(", rpc_time: ")
|
|
buf.WriteString(time.Duration(copRPC.Consume).String())
|
|
}
|
|
buf.WriteString(fmt.Sprintf(", copr_cache_hit_ratio: %v",
|
|
strconv.FormatFloat(float64(s.CoprCacheHitNum)/float64(len(s.copRespTime)), 'f', 2, 64)))
|
|
buf.WriteString("}")
|
|
}
|
|
|
|
rpcStatsStr := rpcStat.String()
|
|
if len(rpcStatsStr) > 0 {
|
|
buf.WriteString(", ")
|
|
buf.WriteString(rpcStatsStr)
|
|
}
|
|
|
|
if len(s.backoffSleep) > 0 {
|
|
buf.WriteString(", backoff{")
|
|
idx := 0
|
|
for k, d := range s.backoffSleep {
|
|
if idx > 0 {
|
|
buf.WriteString(", ")
|
|
}
|
|
idx++
|
|
buf.WriteString(fmt.Sprintf("%s: %s", k, d.String()))
|
|
}
|
|
buf.WriteString("}")
|
|
}
|
|
return buf.String()
|
|
}
|
|
|
|
// Tp implements the RuntimeStats interface.
|
|
func (s *selectResultRuntimeStats) Tp() int {
|
|
return execdetails.TpSelectResultRuntimeStats
|
|
}
|
|
|