443 lines
14 KiB
Go
443 lines
14 KiB
Go
// Copyright 2015 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,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
//go:build !codes
|
|
// +build !codes
|
|
|
|
package testkit
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"fmt"
|
|
"sort"
|
|
"strings"
|
|
"sync/atomic"
|
|
"time"
|
|
|
|
"github.com/pingcap/check"
|
|
"github.com/pingcap/errors"
|
|
"github.com/pingcap/tidb/ddl"
|
|
"github.com/pingcap/tidb/domain"
|
|
"github.com/pingcap/tidb/kv"
|
|
"github.com/pingcap/tidb/parser/model"
|
|
"github.com/pingcap/tidb/parser/terror"
|
|
"github.com/pingcap/tidb/session"
|
|
"github.com/pingcap/tidb/sessionctx/variable"
|
|
"github.com/pingcap/tidb/types"
|
|
"github.com/pingcap/tidb/util/sqlexec"
|
|
"github.com/pingcap/tidb/util/testutil"
|
|
)
|
|
|
|
// TestKit is a utility to run sql test.
|
|
type TestKit struct {
|
|
c *check.C
|
|
store kv.Storage
|
|
Se session.Session
|
|
}
|
|
|
|
// Result is the result returned by MustQuery.
|
|
type Result struct {
|
|
rows [][]string
|
|
comment check.CommentInterface
|
|
c *check.C
|
|
}
|
|
|
|
// Check asserts the result equals the expected results.
|
|
func (res *Result) Check(expected [][]interface{}) {
|
|
resBuff := bytes.NewBufferString("")
|
|
for _, row := range res.rows {
|
|
fmt.Fprintf(resBuff, "%s\n", row)
|
|
}
|
|
needBuff := bytes.NewBufferString("")
|
|
for _, row := range expected {
|
|
fmt.Fprintf(needBuff, "%s\n", row)
|
|
}
|
|
res.c.Assert(resBuff.String(), check.Equals, needBuff.String(), res.comment)
|
|
}
|
|
|
|
// CheckAt asserts the result of selected columns equals the expected results.
|
|
func (res *Result) CheckAt(cols []int, expected [][]interface{}) {
|
|
for _, e := range expected {
|
|
res.c.Assert(len(cols), check.Equals, len(e))
|
|
}
|
|
|
|
rows := make([][]string, 0, len(expected))
|
|
for i := range res.rows {
|
|
row := make([]string, 0, len(cols))
|
|
for _, r := range cols {
|
|
row = append(row, res.rows[i][r])
|
|
}
|
|
rows = append(rows, row)
|
|
}
|
|
got := fmt.Sprintf("%s", rows)
|
|
need := fmt.Sprintf("%s", expected)
|
|
res.c.Assert(got, check.Equals, need, res.comment)
|
|
}
|
|
|
|
// Rows returns the result data.
|
|
func (res *Result) Rows() [][]interface{} {
|
|
ifacesSlice := make([][]interface{}, len(res.rows))
|
|
for i := range res.rows {
|
|
ifaces := make([]interface{}, len(res.rows[i]))
|
|
for j := range res.rows[i] {
|
|
ifaces[j] = res.rows[i][j]
|
|
}
|
|
ifacesSlice[i] = ifaces
|
|
}
|
|
return ifacesSlice
|
|
}
|
|
|
|
// Sort sorts and return the result.
|
|
func (res *Result) Sort() *Result {
|
|
sort.Slice(res.rows, func(i, j int) bool {
|
|
a := res.rows[i]
|
|
b := res.rows[j]
|
|
for i := range a {
|
|
if a[i] < b[i] {
|
|
return true
|
|
} else if a[i] > b[i] {
|
|
return false
|
|
}
|
|
}
|
|
return false
|
|
})
|
|
return res
|
|
}
|
|
|
|
// NewTestKit returns a new *TestKit.
|
|
func NewTestKit(c *check.C, store kv.Storage) *TestKit {
|
|
return &TestKit{
|
|
c: c,
|
|
store: store,
|
|
}
|
|
}
|
|
|
|
// NewTestKitWithSession returns a new *TestKit with a session.
|
|
func NewTestKitWithSession(c *check.C, store kv.Storage, se session.Session) *TestKit {
|
|
return &TestKit{
|
|
c: c,
|
|
store: store,
|
|
Se: se,
|
|
}
|
|
}
|
|
|
|
// NewTestKitWithInit returns a new *TestKit and creates a session.
|
|
func NewTestKitWithInit(c *check.C, store kv.Storage) *TestKit {
|
|
tk := NewTestKit(c, store)
|
|
// Use test and prepare a session.
|
|
tk.MustExec("use test")
|
|
return tk
|
|
}
|
|
|
|
// MockGC is used to make GC work in the test environment.
|
|
func MockGC(tk *TestKit) (string, string, string, func()) {
|
|
originGC := ddl.IsEmulatorGCEnable()
|
|
resetGC := func() {
|
|
if originGC {
|
|
ddl.EmulatorGCEnable()
|
|
} else {
|
|
ddl.EmulatorGCDisable()
|
|
}
|
|
}
|
|
|
|
// disable emulator GC.
|
|
// Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl.
|
|
ddl.EmulatorGCDisable()
|
|
gcTimeFormat := "20060102-15:04:05 -0700 MST"
|
|
timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat)
|
|
timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat)
|
|
safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '')
|
|
ON DUPLICATE KEY
|
|
UPDATE variable_value = '%[1]s'`
|
|
// clear GC variables first.
|
|
tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )")
|
|
return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC
|
|
}
|
|
|
|
var connectionID uint64
|
|
|
|
// GetConnectionID get the connection ID for tk.Se
|
|
func (tk *TestKit) GetConnectionID() {
|
|
if tk.Se != nil {
|
|
id := atomic.AddUint64(&connectionID, 1)
|
|
tk.Se.SetConnectionID(id)
|
|
}
|
|
}
|
|
|
|
// Exec executes a sql statement using the prepared stmt API
|
|
func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, error) {
|
|
var err error
|
|
if tk.Se == nil {
|
|
tk.Se, err = session.CreateSession4Test(tk.store)
|
|
tk.c.Assert(err, check.IsNil)
|
|
tk.GetConnectionID()
|
|
}
|
|
ctx := context.Background()
|
|
if len(args) == 0 {
|
|
sc := tk.Se.GetSessionVars().StmtCtx
|
|
prevWarns := sc.GetWarnings()
|
|
stmts, err := tk.Se.Parse(ctx, sql)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
warns := sc.GetWarnings()
|
|
parserWarns := warns[len(prevWarns):]
|
|
var rs0 sqlexec.RecordSet
|
|
for i, stmt := range stmts {
|
|
rs, err := tk.Se.ExecuteStmt(ctx, stmt)
|
|
if i == 0 {
|
|
rs0 = rs
|
|
}
|
|
if err != nil {
|
|
tk.Se.GetSessionVars().StmtCtx.AppendError(err)
|
|
return nil, errors.Trace(err)
|
|
}
|
|
}
|
|
if len(parserWarns) > 0 {
|
|
tk.Se.GetSessionVars().StmtCtx.AppendWarnings(parserWarns)
|
|
}
|
|
return rs0, nil
|
|
}
|
|
stmtID, _, _, err := tk.Se.PrepareStmt(sql)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
params := make([]types.Datum, len(args))
|
|
for i := 0; i < len(params); i++ {
|
|
params[i] = types.NewDatum(args[i])
|
|
}
|
|
rs, err := tk.Se.ExecutePreparedStmt(ctx, stmtID, params)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
err = tk.Se.DropPreparedStmt(stmtID)
|
|
if err != nil {
|
|
return nil, errors.Trace(err)
|
|
}
|
|
return rs, nil
|
|
}
|
|
|
|
// CheckExecResult checks the affected rows and the insert id after executing MustExec.
|
|
func (tk *TestKit) CheckExecResult(affectedRows, insertID int64) {
|
|
tk.c.Assert(affectedRows, check.Equals, int64(tk.Se.AffectedRows()))
|
|
tk.c.Assert(insertID, check.Equals, int64(tk.Se.LastInsertID()))
|
|
}
|
|
|
|
// CheckLastMessage checks last message after executing MustExec
|
|
func (tk *TestKit) CheckLastMessage(msg string) {
|
|
tk.c.Assert(tk.Se.LastMessage(), check.Equals, msg)
|
|
}
|
|
|
|
// MustExec executes a sql statement and asserts nil error.
|
|
func (tk *TestKit) MustExec(sql string, args ...interface{}) {
|
|
res, err := tk.Exec(sql, args...)
|
|
tk.c.Assert(err, check.IsNil, check.Commentf("sql:%s, %v, error stack %v", sql, args, errors.ErrorStack(err)))
|
|
if res != nil {
|
|
tk.c.Assert(res.Close(), check.IsNil)
|
|
}
|
|
}
|
|
|
|
// HasPlan checks if the result execution plan contains specific plan.
|
|
func (tk *TestKit) HasPlan(sql string, plan string, args ...interface{}) bool {
|
|
rs := tk.MustQuery("explain "+sql, args...)
|
|
for i := range rs.rows {
|
|
if strings.Contains(rs.rows[i][0], plan) {
|
|
return true
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
|
|
func containGlobal(rs *Result) bool {
|
|
partitionNameCol := 2
|
|
for i := range rs.rows {
|
|
if strings.Contains(rs.rows[i][partitionNameCol], "global") {
|
|
return true
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
|
|
// MustNoGlobalStats checks if there is no global stats.
|
|
func (tk *TestKit) MustNoGlobalStats(table string) bool {
|
|
if containGlobal(tk.MustQuery("show stats_meta where table_name like '" + table + "'")) {
|
|
return false
|
|
}
|
|
if containGlobal(tk.MustQuery("show stats_buckets where table_name like '" + table + "'")) {
|
|
return false
|
|
}
|
|
if containGlobal(tk.MustQuery("show stats_histograms where table_name like '" + table + "'")) {
|
|
return false
|
|
}
|
|
return true
|
|
}
|
|
|
|
// MustPartition checks if the result execution plan must read specific partitions.
|
|
func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result {
|
|
rs := tk.MustQuery("explain "+sql, args...)
|
|
ok := len(partitions) == 0
|
|
for i := range rs.rows {
|
|
if len(partitions) == 0 && strings.Contains(rs.rows[i][3], "partition:") {
|
|
ok = false
|
|
}
|
|
if len(partitions) != 0 && strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 {
|
|
ok = true
|
|
}
|
|
}
|
|
tk.c.Assert(ok, check.IsTrue)
|
|
return tk.MustQuery(sql, args...)
|
|
}
|
|
|
|
// UsedPartitions returns the partition names that will be used or all/dual.
|
|
func (tk *TestKit) UsedPartitions(sql string, args ...interface{}) *Result {
|
|
rs := tk.MustQuery("explain "+sql, args...)
|
|
var usedPartitions [][]string
|
|
for i := range rs.rows {
|
|
index := strings.Index(rs.rows[i][3], "partition:")
|
|
if index != -1 {
|
|
p := rs.rows[i][3][index+len("partition:"):]
|
|
partitions := strings.Split(strings.SplitN(p, " ", 2)[0], ",")
|
|
usedPartitions = append(usedPartitions, partitions)
|
|
}
|
|
}
|
|
return &Result{rows: usedPartitions, c: tk.c, comment: check.Commentf("sql:%s, args:%v", sql, args)}
|
|
}
|
|
|
|
// MustUseIndex checks if the result execution plan contains specific index(es).
|
|
func (tk *TestKit) MustUseIndex(sql string, index string, args ...interface{}) bool {
|
|
rs := tk.MustQuery("explain "+sql, args...)
|
|
for i := range rs.rows {
|
|
if strings.Contains(rs.rows[i][3], "index:"+index) {
|
|
return true
|
|
}
|
|
}
|
|
return false
|
|
}
|
|
|
|
// MustIndexLookup checks whether the plan for the sql is IndexLookUp.
|
|
func (tk *TestKit) MustIndexLookup(sql string, args ...interface{}) *Result {
|
|
tk.c.Assert(tk.HasPlan(sql, "IndexLookUp", args...), check.IsTrue)
|
|
return tk.MustQuery(sql, args...)
|
|
}
|
|
|
|
// MustTableDual checks whether the plan for the sql is TableDual.
|
|
func (tk *TestKit) MustTableDual(sql string, args ...interface{}) *Result {
|
|
tk.c.Assert(tk.HasPlan(sql, "TableDual", args...), check.IsTrue)
|
|
return tk.MustQuery(sql, args...)
|
|
}
|
|
|
|
// MustPointGet checks whether the plan for the sql is Point_Get.
|
|
func (tk *TestKit) MustPointGet(sql string, args ...interface{}) *Result {
|
|
rs := tk.MustQuery("explain "+sql, args...)
|
|
tk.c.Assert(len(rs.rows), check.Equals, 1)
|
|
tk.c.Assert(strings.Contains(rs.rows[0][0], "Point_Get"), check.IsTrue, check.Commentf("plan %v", rs.rows[0][0]))
|
|
return tk.MustQuery(sql, args...)
|
|
}
|
|
|
|
// MustQuery query the statements and returns result rows.
|
|
// If expected result is set it asserts the query result equals expected result.
|
|
func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result {
|
|
comment := check.Commentf("sql:%s, args:%v", sql, args)
|
|
rs, err := tk.Exec(sql, args...)
|
|
tk.c.Assert(err, check.IsNil, comment)
|
|
tk.c.Assert(rs, check.NotNil, comment)
|
|
return tk.ResultSetToResult(rs, comment)
|
|
}
|
|
|
|
// MayQuery query the statements and returns result rows if result set is returned.
|
|
// If expected result is set it asserts the query result equals expected result.
|
|
func (tk *TestKit) MayQuery(sql string, args ...interface{}) *Result {
|
|
comment := check.Commentf("sql:%s, args:%v", sql, args)
|
|
rs, err := tk.Exec(sql, args...)
|
|
tk.c.Assert(err, check.IsNil, comment)
|
|
if rs == nil {
|
|
var emptyStringAoA [][]string
|
|
return &Result{rows: emptyStringAoA, c: tk.c, comment: comment}
|
|
}
|
|
return tk.ResultSetToResult(rs, comment)
|
|
}
|
|
|
|
// QueryToErr executes a sql statement and discard results.
|
|
func (tk *TestKit) QueryToErr(sql string, args ...interface{}) error {
|
|
comment := check.Commentf("sql:%s, args:%v", sql, args)
|
|
res, err := tk.Exec(sql, args...)
|
|
tk.c.Assert(err, check.IsNil, comment)
|
|
tk.c.Assert(res, check.NotNil, comment)
|
|
_, resErr := session.GetRows4Test(context.Background(), tk.Se, res)
|
|
tk.c.Assert(res.Close(), check.IsNil)
|
|
return resErr
|
|
}
|
|
|
|
// ExecToErr executes a sql statement and discard results.
|
|
func (tk *TestKit) ExecToErr(sql string, args ...interface{}) error {
|
|
res, err := tk.Exec(sql, args...)
|
|
if res != nil {
|
|
tk.c.Assert(res.Close(), check.IsNil)
|
|
}
|
|
return err
|
|
}
|
|
|
|
// MustGetErrMsg executes a sql statement and assert it's error message.
|
|
func (tk *TestKit) MustGetErrMsg(sql string, errStr string) {
|
|
err := tk.ExecToErr(sql)
|
|
tk.c.Assert(err, check.NotNil)
|
|
tk.c.Assert(err.Error(), check.Equals, errStr)
|
|
}
|
|
|
|
// MustGetErrCode executes a sql statement and assert it's error code.
|
|
func (tk *TestKit) MustGetErrCode(sql string, errCode int) {
|
|
_, err := tk.Exec(sql)
|
|
tk.c.Assert(err, check.NotNil)
|
|
originErr := errors.Cause(err)
|
|
tErr, ok := originErr.(*terror.Error)
|
|
tk.c.Assert(ok, check.IsTrue, check.Commentf("expect type 'terror.Error', but obtain '%T': %v", originErr, originErr))
|
|
sqlErr := terror.ToSQLError(tErr)
|
|
tk.c.Assert(int(sqlErr.Code), check.Equals, errCode, check.Commentf("Assertion failed, origin err:\n %v", sqlErr))
|
|
}
|
|
|
|
// ResultSetToResult converts sqlexec.RecordSet to testkit.Result.
|
|
// It is used to check results of execute statement in binary mode.
|
|
func (tk *TestKit) ResultSetToResult(rs sqlexec.RecordSet, comment check.CommentInterface) *Result {
|
|
return tk.ResultSetToResultWithCtx(context.Background(), rs, comment)
|
|
}
|
|
|
|
// ResultSetToResultWithCtx converts sqlexec.RecordSet to testkit.Result.
|
|
func (tk *TestKit) ResultSetToResultWithCtx(ctx context.Context, rs sqlexec.RecordSet, comment check.CommentInterface) *Result {
|
|
sRows, err := session.ResultSetToStringSlice(ctx, tk.Se, rs)
|
|
tk.c.Check(err, check.IsNil, comment)
|
|
return &Result{rows: sRows, c: tk.c, comment: comment}
|
|
}
|
|
|
|
// Rows is similar to RowsWithSep, use white space as separator string.
|
|
func Rows(args ...string) [][]interface{} {
|
|
return testutil.RowsWithSep(" ", args...)
|
|
}
|
|
|
|
// GetTableID gets table ID by name.
|
|
func (tk *TestKit) GetTableID(tableName string) int64 {
|
|
dom := domain.GetDomain(tk.Se)
|
|
is := dom.InfoSchema()
|
|
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(tableName))
|
|
tk.c.Assert(err, check.IsNil)
|
|
return tbl.Meta().ID
|
|
}
|
|
|
|
// WithPruneMode run test case under prune mode.
|
|
func WithPruneMode(tk *TestKit, mode variable.PartitionPruneMode, f func()) {
|
|
tk.MustExec("set @@tidb_partition_prune_mode=`" + string(mode) + "`")
|
|
tk.MustExec("set global tidb_partition_prune_mode=`" + string(mode) + "`")
|
|
f()
|
|
}
|