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.
551 lines
15 KiB
551 lines
15 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 (
|
|
"context"
|
|
"sync"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/cznic/mathutil"
|
|
. "github.com/pingcap/check"
|
|
"github.com/pingcap/parser/charset"
|
|
"github.com/pingcap/parser/mysql"
|
|
"github.com/pingcap/tidb/kv"
|
|
"github.com/pingcap/tidb/sessionctx"
|
|
"github.com/pingcap/tidb/sessionctx/variable"
|
|
"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/execdetails"
|
|
"github.com/pingcap/tidb/util/memory"
|
|
"github.com/pingcap/tipb/go-tipb"
|
|
)
|
|
|
|
func (s *testSuite) createSelectNormal(batch, totalRows int, c *C, planIDs []int) (*selectResult, []*types.FieldType) {
|
|
request, err := (&RequestBuilder{}).SetKeyRanges(nil).
|
|
SetDAGRequest(&tipb.DAGRequest{}).
|
|
SetDesc(false).
|
|
SetKeepOrder(false).
|
|
SetFromSessionVars(variable.NewSessionVars()).
|
|
SetMemTracker(memory.NewTracker(-1, -1)).
|
|
Build()
|
|
c.Assert(err, IsNil)
|
|
|
|
/// 4 int64 types.
|
|
colTypes := []*types.FieldType{
|
|
{
|
|
Tp: mysql.TypeLonglong,
|
|
Flen: mysql.MaxIntWidth,
|
|
Decimal: 0,
|
|
Flag: mysql.BinaryFlag,
|
|
Charset: charset.CharsetBin,
|
|
Collate: charset.CollationBin,
|
|
},
|
|
}
|
|
colTypes = append(colTypes, colTypes[0])
|
|
colTypes = append(colTypes, colTypes[0])
|
|
colTypes = append(colTypes, colTypes[0])
|
|
|
|
// Test Next.
|
|
var response SelectResult
|
|
if planIDs == nil {
|
|
response, err = Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false))
|
|
} else {
|
|
response, err = SelectWithRuntimeStats(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false), planIDs, 1)
|
|
}
|
|
|
|
c.Assert(err, IsNil)
|
|
result, ok := response.(*selectResult)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(result.label, Equals, "dag")
|
|
c.Assert(result.sqlType, Equals, "general")
|
|
c.Assert(result.rowLen, Equals, len(colTypes))
|
|
|
|
resp, ok := result.resp.(*mockResponse)
|
|
c.Assert(ok, IsTrue)
|
|
resp.total = totalRows
|
|
resp.batch = batch
|
|
|
|
return result, colTypes
|
|
}
|
|
|
|
func (s *testSuite) TestSelectNormal(c *C) {
|
|
response, colTypes := s.createSelectNormal(1, 2, c, nil)
|
|
response.Fetch(context.TODO())
|
|
|
|
// Test Next.
|
|
chk := chunk.New(colTypes, 32, 32)
|
|
numAllRows := 0
|
|
for {
|
|
err := response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
numAllRows += chk.NumRows()
|
|
if chk.NumRows() == 0 {
|
|
break
|
|
}
|
|
}
|
|
c.Assert(numAllRows, Equals, 2)
|
|
err := response.Close()
|
|
c.Assert(err, IsNil)
|
|
c.Assert(response.memTracker.BytesConsumed(), Equals, int64(0))
|
|
}
|
|
|
|
func (s *testSuite) TestSelectMemTracker(c *C) {
|
|
response, colTypes := s.createSelectNormal(2, 6, c, nil)
|
|
response.Fetch(context.TODO())
|
|
|
|
// Test Next.
|
|
chk := chunk.New(colTypes, 3, 3)
|
|
err := response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.IsFull(), Equals, true)
|
|
err = response.Close()
|
|
c.Assert(err, IsNil)
|
|
c.Assert(response.memTracker.BytesConsumed(), Equals, int64(0))
|
|
}
|
|
|
|
func (s *testSuite) TestSelectNormalChunkSize(c *C) {
|
|
s.sctx.GetSessionVars().EnableChunkRPC = false
|
|
response, colTypes := s.createSelectNormal(100, 1000000, c, nil)
|
|
response.Fetch(context.TODO())
|
|
s.testChunkSize(response, colTypes, c)
|
|
c.Assert(response.Close(), IsNil)
|
|
c.Assert(response.memTracker.BytesConsumed(), Equals, int64(0))
|
|
}
|
|
|
|
func (s *testSuite) TestSelectWithRuntimeStats(c *C) {
|
|
planIDs := []int{1, 2, 3}
|
|
response, colTypes := s.createSelectNormal(1, 2, c, planIDs)
|
|
if len(response.copPlanIDs) != len(planIDs) {
|
|
c.Fatal("invalid copPlanIDs")
|
|
}
|
|
for i := range planIDs {
|
|
if response.copPlanIDs[i] != planIDs[i] {
|
|
c.Fatal("invalid copPlanIDs")
|
|
}
|
|
}
|
|
|
|
response.Fetch(context.TODO())
|
|
|
|
// Test Next.
|
|
chk := chunk.New(colTypes, 32, 32)
|
|
numAllRows := 0
|
|
for {
|
|
err := response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
numAllRows += chk.NumRows()
|
|
if chk.NumRows() == 0 {
|
|
break
|
|
}
|
|
}
|
|
c.Assert(numAllRows, Equals, 2)
|
|
err := response.Close()
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
func (s *testSuite) TestSelectResultRuntimeStats(c *C) {
|
|
basic := &execdetails.BasicRuntimeStats{}
|
|
basic.Record(time.Second, 20)
|
|
s1 := &selectResultRuntimeStats{
|
|
copRespTime: []time.Duration{time.Second, time.Millisecond},
|
|
procKeys: []int64{100, 200},
|
|
backoffSleep: map[string]time.Duration{"RegionMiss": time.Millisecond},
|
|
totalProcessTime: time.Second,
|
|
totalWaitTime: time.Second,
|
|
rpcStat: tikv.NewRegionRequestRuntimeStats(),
|
|
}
|
|
s2 := *s1
|
|
stmtStats := execdetails.NewRuntimeStatsColl()
|
|
stmtStats.RegisterStats(1, basic)
|
|
stmtStats.RegisterStats(1, s1)
|
|
stmtStats.RegisterStats(1, &s2)
|
|
stats := stmtStats.GetRootStats(1)
|
|
expect := "time:1s, loops:1, cop_task: {num: 4, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 2s, tot_wait: 2s, copr_cache_hit_ratio: 0.00}, backoff{RegionMiss: 2ms}"
|
|
c.Assert(stats.String(), Equals, expect)
|
|
// Test for idempotence.
|
|
c.Assert(stats.String(), Equals, expect)
|
|
|
|
s1.rpcStat.Stats[tikvrpc.CmdCop] = &tikv.RPCRuntimeStats{
|
|
Count: 1,
|
|
Consume: int64(time.Second),
|
|
}
|
|
stmtStats.RegisterStats(2, s1)
|
|
stats = stmtStats.GetRootStats(2)
|
|
expect = "cop_task: {num: 2, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 1s, tot_wait: 1s, rpc_num: 1, rpc_time: 1s, copr_cache_hit_ratio: 0.00}, backoff{RegionMiss: 1ms}"
|
|
c.Assert(stats.String(), Equals, expect)
|
|
// Test for idempotence.
|
|
c.Assert(stats.String(), Equals, expect)
|
|
}
|
|
|
|
func (s *testSuite) createSelectStreaming(batch, totalRows int, c *C) (*streamResult, []*types.FieldType) {
|
|
request, err := (&RequestBuilder{}).SetKeyRanges(nil).
|
|
SetDAGRequest(&tipb.DAGRequest{}).
|
|
SetDesc(false).
|
|
SetKeepOrder(false).
|
|
SetFromSessionVars(variable.NewSessionVars()).
|
|
SetStreaming(true).
|
|
Build()
|
|
c.Assert(err, IsNil)
|
|
|
|
/// 4 int64 types.
|
|
colTypes := []*types.FieldType{
|
|
{
|
|
Tp: mysql.TypeLonglong,
|
|
Flen: mysql.MaxIntWidth,
|
|
Decimal: 0,
|
|
Flag: mysql.BinaryFlag,
|
|
Charset: charset.CharsetBin,
|
|
Collate: charset.CollationBin,
|
|
},
|
|
}
|
|
colTypes = append(colTypes, colTypes[0])
|
|
colTypes = append(colTypes, colTypes[0])
|
|
colTypes = append(colTypes, colTypes[0])
|
|
|
|
s.sctx.GetSessionVars().EnableStreaming = true
|
|
|
|
response, err := Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false))
|
|
c.Assert(err, IsNil)
|
|
result, ok := response.(*streamResult)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(result.rowLen, Equals, len(colTypes))
|
|
|
|
resp, ok := result.resp.(*mockResponse)
|
|
c.Assert(ok, IsTrue)
|
|
resp.total = totalRows
|
|
resp.batch = batch
|
|
|
|
return result, colTypes
|
|
}
|
|
|
|
func (s *testSuite) TestSelectStreaming(c *C) {
|
|
response, colTypes := s.createSelectStreaming(1, 2, c)
|
|
response.Fetch(context.TODO())
|
|
|
|
// Test Next.
|
|
chk := chunk.New(colTypes, 32, 32)
|
|
numAllRows := 0
|
|
for {
|
|
err := response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
numAllRows += chk.NumRows()
|
|
if chk.NumRows() == 0 {
|
|
break
|
|
}
|
|
}
|
|
c.Assert(numAllRows, Equals, 2)
|
|
err := response.Close()
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
func (s *testSuite) TestSelectStreamingWithNextRaw(c *C) {
|
|
response, _ := s.createSelectStreaming(1, 2, c)
|
|
response.Fetch(context.TODO())
|
|
data, err := response.NextRaw(context.TODO())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(len(data), Equals, 16)
|
|
}
|
|
|
|
func (s *testSuite) TestSelectStreamingChunkSize(c *C) {
|
|
response, colTypes := s.createSelectStreaming(100, 1000000, c)
|
|
response.Fetch(context.TODO())
|
|
s.testChunkSize(response, colTypes, c)
|
|
c.Assert(response.Close(), IsNil)
|
|
}
|
|
|
|
func (s *testSuite) testChunkSize(response SelectResult, colTypes []*types.FieldType, c *C) {
|
|
chk := chunk.New(colTypes, 32, 32)
|
|
|
|
err := response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 32)
|
|
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 32)
|
|
|
|
chk.SetRequiredRows(1, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 1)
|
|
|
|
chk.SetRequiredRows(2, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 2)
|
|
|
|
chk.SetRequiredRows(17, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 17)
|
|
|
|
chk.SetRequiredRows(170, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 32)
|
|
|
|
chk.SetRequiredRows(32, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 32)
|
|
|
|
chk.SetRequiredRows(0, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 32)
|
|
|
|
chk.SetRequiredRows(-1, 32)
|
|
err = response.Next(context.TODO(), chk)
|
|
c.Assert(err, IsNil)
|
|
c.Assert(chk.NumRows(), Equals, 32)
|
|
}
|
|
|
|
func (s *testSuite) TestAnalyze(c *C) {
|
|
s.sctx.GetSessionVars().EnableChunkRPC = false
|
|
request, err := (&RequestBuilder{}).SetKeyRanges(nil).
|
|
SetAnalyzeRequest(&tipb.AnalyzeReq{}).
|
|
SetKeepOrder(true).
|
|
Build()
|
|
c.Assert(err, IsNil)
|
|
|
|
response, err := Analyze(context.TODO(), s.sctx.GetClient(), request, kv.DefaultVars, true, s.sctx.GetSessionVars().StmtCtx.MemTracker)
|
|
c.Assert(err, IsNil)
|
|
|
|
result, ok := response.(*selectResult)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(result.label, Equals, "analyze")
|
|
c.Assert(result.sqlType, Equals, "internal")
|
|
|
|
response.Fetch(context.TODO())
|
|
|
|
bytes, err := response.NextRaw(context.TODO())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(len(bytes), Equals, 16)
|
|
|
|
err = response.Close()
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
func (s *testSuite) TestChecksum(c *C) {
|
|
s.sctx.GetSessionVars().EnableChunkRPC = false
|
|
request, err := (&RequestBuilder{}).SetKeyRanges(nil).
|
|
SetChecksumRequest(&tipb.ChecksumRequest{}).
|
|
Build()
|
|
c.Assert(err, IsNil)
|
|
|
|
response, err := Checksum(context.TODO(), s.sctx.GetClient(), request, kv.DefaultVars)
|
|
c.Assert(err, IsNil)
|
|
|
|
result, ok := response.(*selectResult)
|
|
c.Assert(ok, IsTrue)
|
|
c.Assert(result.label, Equals, "checksum")
|
|
c.Assert(result.sqlType, Equals, "general")
|
|
|
|
response.Fetch(context.TODO())
|
|
|
|
bytes, err := response.NextRaw(context.TODO())
|
|
c.Assert(err, IsNil)
|
|
c.Assert(len(bytes), Equals, 16)
|
|
|
|
err = response.Close()
|
|
c.Assert(err, IsNil)
|
|
}
|
|
|
|
// mockResponse implements kv.Response interface.
|
|
// Used only for test.
|
|
type mockResponse struct {
|
|
count int
|
|
total int
|
|
batch int
|
|
ctx sessionctx.Context
|
|
sync.Mutex
|
|
}
|
|
|
|
// Close implements kv.Response interface.
|
|
func (resp *mockResponse) Close() error {
|
|
resp.Lock()
|
|
defer resp.Unlock()
|
|
|
|
resp.count = 0
|
|
return nil
|
|
}
|
|
|
|
// Next implements kv.Response interface.
|
|
func (resp *mockResponse) Next(ctx context.Context) (kv.ResultSubset, error) {
|
|
resp.Lock()
|
|
defer resp.Unlock()
|
|
|
|
if resp.count >= resp.total {
|
|
return nil, nil
|
|
}
|
|
numRows := mathutil.Min(resp.batch, resp.total-resp.count)
|
|
resp.count += numRows
|
|
|
|
var chunks []tipb.Chunk
|
|
if !canUseChunkRPC(resp.ctx) {
|
|
datum := types.NewIntDatum(1)
|
|
bytes := make([]byte, 0, 100)
|
|
bytes, _ = codec.EncodeValue(nil, bytes, datum, datum, datum, datum)
|
|
chunks = make([]tipb.Chunk, numRows)
|
|
for i := range chunks {
|
|
chkData := make([]byte, len(bytes))
|
|
copy(chkData, bytes)
|
|
chunks[i] = tipb.Chunk{RowsData: chkData}
|
|
}
|
|
} else {
|
|
chunks = make([]tipb.Chunk, 0)
|
|
for numRows > 0 {
|
|
rows := mathutil.Min(numRows, 1024)
|
|
numRows -= rows
|
|
|
|
colTypes := make([]*types.FieldType, 4)
|
|
for i := 0; i < 4; i++ {
|
|
colTypes[i] = &types.FieldType{Tp: mysql.TypeLonglong}
|
|
}
|
|
chk := chunk.New(colTypes, numRows, numRows)
|
|
|
|
for rowOrdinal := 0; rowOrdinal < rows; rowOrdinal++ {
|
|
for colOrdinal := 0; colOrdinal < 4; colOrdinal++ {
|
|
chk.AppendInt64(colOrdinal, 123)
|
|
}
|
|
}
|
|
|
|
codec := chunk.NewCodec(colTypes)
|
|
buffer := codec.Encode(chk)
|
|
chunks = append(chunks, tipb.Chunk{RowsData: buffer})
|
|
}
|
|
}
|
|
|
|
respPB := &tipb.SelectResponse{
|
|
Chunks: chunks,
|
|
OutputCounts: []int64{1},
|
|
}
|
|
if canUseChunkRPC(resp.ctx) {
|
|
respPB.EncodeType = tipb.EncodeType_TypeChunk
|
|
} else {
|
|
respPB.EncodeType = tipb.EncodeType_TypeDefault
|
|
}
|
|
respBytes, err := respPB.Marshal()
|
|
if err != nil {
|
|
panic(err)
|
|
}
|
|
return &mockResultSubset{respBytes}, nil
|
|
}
|
|
|
|
// mockResultSubset implements kv.ResultSubset interface.
|
|
// Used only for test.
|
|
type mockResultSubset struct{ data []byte }
|
|
|
|
// GetData implements kv.ResultSubset interface.
|
|
func (r *mockResultSubset) GetData() []byte { return r.data }
|
|
|
|
// GetStartKey implements kv.ResultSubset interface.
|
|
func (r *mockResultSubset) GetStartKey() kv.Key { return nil }
|
|
|
|
// MemSize implements kv.ResultSubset interface.
|
|
func (r *mockResultSubset) MemSize() int64 { return int64(cap(r.data)) }
|
|
|
|
// RespTime implements kv.ResultSubset interface.
|
|
func (r *mockResultSubset) RespTime() time.Duration { return 0 }
|
|
|
|
func createSelectNormal(batch, totalRows int, ctx sessionctx.Context) (*selectResult, []*types.FieldType) {
|
|
request, _ := (&RequestBuilder{}).SetKeyRanges(nil).
|
|
SetDAGRequest(&tipb.DAGRequest{}).
|
|
SetDesc(false).
|
|
SetKeepOrder(false).
|
|
SetFromSessionVars(variable.NewSessionVars()).
|
|
SetMemTracker(memory.NewTracker(-1, -1)).
|
|
Build()
|
|
|
|
/// 4 int64 types.
|
|
colTypes := []*types.FieldType{
|
|
{
|
|
Tp: mysql.TypeLonglong,
|
|
Flen: mysql.MaxIntWidth,
|
|
Decimal: 0,
|
|
Flag: mysql.BinaryFlag,
|
|
Charset: charset.CharsetBin,
|
|
Collate: charset.CollationBin,
|
|
},
|
|
}
|
|
colTypes = append(colTypes, colTypes[0])
|
|
colTypes = append(colTypes, colTypes[0])
|
|
colTypes = append(colTypes, colTypes[0])
|
|
|
|
// Test Next.
|
|
var response SelectResult
|
|
response, _ = Select(context.TODO(), ctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false))
|
|
|
|
result, _ := response.(*selectResult)
|
|
resp, _ := result.resp.(*mockResponse)
|
|
resp.total = totalRows
|
|
resp.batch = batch
|
|
|
|
return result, colTypes
|
|
}
|
|
|
|
func BenchmarkSelectResponseChunk_BigResponse(b *testing.B) {
|
|
for i := 0; i < b.N; i++ {
|
|
b.StopTimer()
|
|
s := &testSuite{}
|
|
s.SetUpSuite(nil)
|
|
s.sctx.GetSessionVars().InitChunkSize = 32
|
|
s.sctx.GetSessionVars().MaxChunkSize = 1024
|
|
selectResult, colTypes := createSelectNormal(4000, 20000, s.sctx)
|
|
selectResult.Fetch(context.TODO())
|
|
chk := chunk.NewChunkWithCapacity(colTypes, 1024)
|
|
b.StartTimer()
|
|
for {
|
|
err := selectResult.Next(context.TODO(), chk)
|
|
if err != nil {
|
|
panic(err)
|
|
}
|
|
if chk.NumRows() == 0 {
|
|
break
|
|
}
|
|
chk.Reset()
|
|
}
|
|
s.TearDownSuite(nil)
|
|
}
|
|
}
|
|
|
|
func BenchmarkSelectResponseChunk_SmallResponse(b *testing.B) {
|
|
for i := 0; i < b.N; i++ {
|
|
b.StopTimer()
|
|
s := &testSuite{}
|
|
s.SetUpSuite(nil)
|
|
s.sctx.GetSessionVars().InitChunkSize = 32
|
|
s.sctx.GetSessionVars().MaxChunkSize = 1024
|
|
selectResult, colTypes := createSelectNormal(32, 3200, s.sctx)
|
|
selectResult.Fetch(context.TODO())
|
|
chk := chunk.NewChunkWithCapacity(colTypes, 1024)
|
|
b.StartTimer()
|
|
for {
|
|
err := selectResult.Next(context.TODO(), chk)
|
|
if err != nil {
|
|
panic(err)
|
|
}
|
|
if chk.NumRows() == 0 {
|
|
break
|
|
}
|
|
chk.Reset()
|
|
}
|
|
s.TearDownSuite(nil)
|
|
}
|
|
}
|
|
|