Skip to content

Commit

Permalink
migrate test-infra to testify for distsql_test.go pingcap#28574
Browse files Browse the repository at this point in the history
  • Loading branch information
znhh6018 committed Dec 25, 2021
1 parent 76aae0d commit c57e629
Showing 1 changed file with 123 additions and 71 deletions.
194 changes: 123 additions & 71 deletions executor/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,23 +21,26 @@ import (
"math/rand"
"runtime/pprof"
"strings"
"testing"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/copr"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/israce"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/testutils"
)

// nolint:unused
func checkGoroutineExists(keyword string) bool {
buf := new(bytes.Buffer)
profile := pprof.Lookup("goroutine")
Expand All @@ -49,13 +52,21 @@ func checkGoroutineExists(keyword string) bool {
return strings.Contains(str, keyword)
}

func (s *testSuite3) TestCopClientSend(c *C) {
c.Skip("not stable")
if _, ok := s.store.GetClient().(*copr.CopClient); !ok {
func TestCopClientSend(t *testing.T) {
t.Parallel()
t.Skip("not stable")
var cluster testutils.Cluster
store, clean := testkit.CreateMockStore(t, mockstore.WithClusterInspector(func(c testutils.Cluster) {
mockstore.BootstrapWithSingleStore(c)
cluster = c
}))
defer clean()
if _, ok := store.GetClient().(*copr.CopClient); !ok {
// Make sure the store is tikv store.
return
}
tk := testkit.NewTestKit(c, s.store)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("create table copclient (id int primary key)")

Expand All @@ -67,53 +78,53 @@ func (s *testSuite3) TestCopClientSend(c *C) {
tk.MustExec("insert copclient values " + strings.Join(values, ","))

// Get table ID for split.
dom := domain.GetDomain(tk.Se)
dom := domain.GetDomain(tk.Session())
is := dom.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("copclient"))
c.Assert(err, IsNil)
require.NoError(t, err)
tblID := tbl.Meta().ID

// Split the table.
tableStart := tablecodec.GenTableRecordPrefix(tblID)
s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100)
cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100)

ctx := context.Background()
// Send coprocessor request when the table split.
rs, err := tk.Exec("select sum(id) from copclient")
c.Assert(err, IsNil)
require.NoError(t, err)
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(req.GetRow(0).GetMyDecimal(0).String(), Equals, "499500")
c.Assert(rs.Close(), IsNil)
require.NoError(t, err)
require.Equal(t, "499500", req.GetRow(0).GetMyDecimal(0).String())
require.NoError(t, rs.Close())

// Split one region.
key := tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(500))
region, _ := s.cluster.GetRegionByKey(key)
peerID := s.cluster.AllocID()
s.cluster.Split(region.GetId(), s.cluster.AllocID(), key, []uint64{peerID}, peerID)
region, _ := cluster.GetRegionByKey(key)
peerID := cluster.AllocID()
cluster.Split(region.GetId(), cluster.AllocID(), key, []uint64{peerID}, peerID)

// Check again.
rs, err = tk.Exec("select sum(id) from copclient")
c.Assert(err, IsNil)
require.NoError(t, err)
req = rs.NewChunk(nil)
err = rs.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(req.GetRow(0).GetMyDecimal(0).String(), Equals, "499500")
c.Assert(rs.Close(), IsNil)
require.NoError(t, err)
require.Equal(t, "499500", req.GetRow(0).GetMyDecimal(0).String())
require.NoError(t, rs.Close())

// Check there is no goroutine leak.
rs, err = tk.Exec("select * from copclient order by id")
c.Assert(err, IsNil)
require.NoError(t, err)
req = rs.NewChunk(nil)
err = rs.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(rs.Close(), IsNil)
require.NoError(t, err)
require.NoError(t, rs.Close())
keyword := "(*copIterator).work"
c.Check(checkGoroutineExists(keyword), IsFalse)
require.False(t, checkGoroutineExists(keyword))
}

func (s *testSuite3) TestGetLackHandles(c *C) {
func TestGetLackHandles(t *testing.T) {
expectedHandles := []kv.Handle{kv.IntHandle(1), kv.IntHandle(2), kv.IntHandle(3), kv.IntHandle(4),
kv.IntHandle(5), kv.IntHandle(6), kv.IntHandle(7), kv.IntHandle(8), kv.IntHandle(9), kv.IntHandle(10)}
handlesMap := kv.NewHandleMap()
Expand All @@ -124,8 +135,8 @@ func (s *testSuite3) TestGetLackHandles(c *C) {
// expected handles 1, 2, 3, 4, 5, 6, 7, 8, 9, 10
// obtained handles 1, 2, 3, 4, 5, 6, 7, 8, 9, 10
diffHandles := executor.GetLackHandles(expectedHandles, handlesMap)
c.Assert(diffHandles, HasLen, 0)
c.Assert(handlesMap.Len(), Equals, 0)
require.Len(t, diffHandles, 0)
require.Equal(t, 0, handlesMap.Len())

// expected handles 1, 2, 3, 4, 5, 6, 7, 8, 9, 10
// obtained handles 2, 3, 4, 6, 7, 8, 9
Expand All @@ -136,19 +147,27 @@ func (s *testSuite3) TestGetLackHandles(c *C) {
handlesMap.Set(kv.IntHandle(5), true)
handlesMap.Set(kv.IntHandle(10), true)
diffHandles = executor.GetLackHandles(expectedHandles, handlesMap)
c.Assert(retHandles, DeepEquals, diffHandles)
require.Equal(t, diffHandles, retHandles) // deep equal
}

func (s *testSuite3) TestBigIntPK(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestBigIntPK(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("create table t(a bigint unsigned primary key, b int, c int, index idx(a, b))")
tk.MustExec("insert into t values(1, 1, 1), (9223372036854775807, 2, 2)")
tk.MustQuery("select * from t use index(idx) order by a").Check(testkit.Rows("1 1 1", "9223372036854775807 2 2"))
}

func (s *testSuite3) TestCorColToRanges(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestCorColToRanges(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only-full-group-by
tk.MustExec("drop table if exists t")
Expand All @@ -163,8 +182,12 @@ func (s *testSuite3) TestCorColToRanges(c *C) {
tk.MustQuery("select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.c = t1.a) from t order by 1 desc").Check(testkit.Rows("1", "0", "0", "0", "0", "0", "0", "0", "0"))
}

func (s *testSuiteP1) TestUniqueKeyNullValueSelect(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestUniqueKeyNullValueSelect(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
// test null in unique-key
Expand All @@ -189,8 +212,12 @@ func (s *testSuiteP1) TestUniqueKeyNullValueSelect(c *C) {
}

// TestIssue10178 contains tests for https://github.com/pingcap/tidb/issues/10178 .
func (s *testSuite3) TestIssue10178(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestIssue10178(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a bigint unsigned primary key)")
Expand All @@ -200,58 +227,67 @@ func (s *testSuite3) TestIssue10178(c *C) {
tk.MustQuery("select * from t where a < 9223372036854775808").Check(testkit.Rows("9223372036854775807"))
}

func (s *testSuite3) TestInconsistentIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestInconsistentIndex(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
dom, err := session.BootstrapSession(store)
require.NoError(t, err)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, index idx_a(a))")
is := s.domain.InfoSchema()
is := dom.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
require.NoError(t, err)
idx := tbl.Meta().FindIndexByName("idx_a")
idxOp := tables.NewIndex(tbl.Meta().ID, tbl.Meta(), idx)
ctx := mock.NewContext()
ctx.Store = s.store
ctx.Store = store

for i := 0; i < 10; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i+10, i))
c.Assert(tk.QueryToErr("select * from t where a>=0"), IsNil)
require.NoError(t, tk.QueryToErr("select * from t where a>=0"))
}

for i := 0; i < 10; i++ {
tk.MustExec(fmt.Sprintf("update t set a=%d where a=%d", i, i+10))
c.Assert(tk.QueryToErr("select * from t where a>=0"), IsNil)
require.NoError(t, tk.QueryToErr("select * from t where a>=0"))
}

for i := 0; i < 10; i++ {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn, err := store.Begin()
require.NoError(t, err)
_, err = idxOp.Create(ctx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i), nil)
c.Assert(err, IsNil)
require.NoError(t, err)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
require.NoError(t, err)

err = tk.QueryToErr("select * from t use index(idx_a) where a >= 0")
c.Assert(err.Error(), Equals, fmt.Sprintf("inconsistent index idx_a handle count %d isn't equal to value count 10", i+11))

require.Equal(t, fmt.Sprintf("inconsistent index idx_a handle count %d isn't equal to value count 10", i+11), err.Error())
// if has other conditions, the inconsistent index check doesn't work.
err = tk.QueryToErr("select * from t where a>=0 and b<10")
c.Assert(err, IsNil)
require.NoError(t, err)
}

// fix inconsistent problem to pass CI
for i := 0; i < 10; i++ {
txn, err := s.store.Begin()
c.Assert(err, IsNil)
txn, err := store.Begin()
require.NoError(t, err)
err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i))
c.Assert(err, IsNil)
require.NoError(t, err)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
require.NoError(t, err)
}
}

func (s *testSuite3) TestPushLimitDownIndexLookUpReader(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestPushLimitDownIndexLookUpReader(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists tbl")
tk.MustExec("create table tbl(a int, b int, c int, key idx_b_c(b,c))")
Expand All @@ -265,8 +301,12 @@ func (s *testSuite3) TestPushLimitDownIndexLookUpReader(c *C) {
tk.MustQuery("select * from tbl use index(idx_b_c) where b > 1 and c > 1 limit 2,1").Check(testkit.Rows("4 4 4"))
}

func (s *testSuite3) TestPartitionTableIndexLookUpReader(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestPartitionTableIndexLookUpReader(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec(`create table t (a int, b int, key(a))
Expand All @@ -289,11 +329,15 @@ func (s *testSuite3) TestPartitionTableIndexLookUpReader(c *C) {
tk.MustQuery("select * from t where a>=1 and a<15 order by a limit 3").Check(testkit.Rows("1 1", "2 2", "11 11"))
}

func (s *testSuite3) TestPartitionTableRandomlyIndexLookUpReader(c *C) {
func TestPartitionTableRandomlyIndexLookUpReader(t *testing.T) {
if israce.RaceEnabled {
c.Skip("exhaustive types test, skip race test")
t.Skip("exhaustive types test, skip race test")
}
tk := testkit.NewTestKit(c, s.store)
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec(`create table t (a int, b int, key(a))
Expand Down Expand Up @@ -326,7 +370,7 @@ func (s *testSuite3) TestPartitionTableRandomlyIndexLookUpReader(c *C) {
}
}

func (s *testSuite3) TestIndexLookUpStats(c *C) {
func TestIndexLookUpStats(t *testing.T) {
stats := &executor.IndexLookUpRunTimeStats{
FetchHandleTotal: int64(5 * time.Second),
FetchHandle: int64(2 * time.Second),
Expand All @@ -335,14 +379,18 @@ func (s *testSuite3) TestIndexLookUpStats(c *C) {
TableTaskNum: 2,
Concurrency: 1,
}
c.Assert(stats.String(), Equals, "index_task: {total_time: 5s, fetch_handle: 2s, build: 1s, wait: 2s}, table_task: {total_time: 2s, num: 2, concurrency: 1}")
c.Assert(stats.String(), Equals, stats.Clone().String())
require.Equal(t, "index_task: {total_time: 5s, fetch_handle: 2s, build: 1s, wait: 2s}, table_task: {total_time: 2s, num: 2, concurrency: 1}", stats.String())
require.Equal(t, stats.Clone().String(), stats.String())
stats.Merge(stats.Clone())
c.Assert(stats.String(), Equals, "index_task: {total_time: 10s, fetch_handle: 4s, build: 2s, wait: 4s}, table_task: {total_time: 4s, num: 4, concurrency: 2}")
require.Equal(t, "index_task: {total_time: 10s, fetch_handle: 4s, build: 2s, wait: 4s}, table_task: {total_time: 4s, num: 4, concurrency: 2}", stats.String())
}

func (s *testSuite3) TestIndexLookUpGetResultChunk(c *C) {
tk := testkit.NewTestKit(c, s.store)
func TestIndexLookUpGetResultChunk(t *testing.T) {
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists tbl")
tk.MustExec("create table tbl(a int, b int, c int, key idx_a(a))")
Expand All @@ -353,11 +401,15 @@ func (s *testSuite3) TestIndexLookUpGetResultChunk(c *C) {
tk.MustQuery("select * from tbl use index(idx_a) where a > 10 order by a asc limit 4,1").Check(testkit.Rows("15 15 15"))
}

func (s *testSuite3) TestPartitionTableIndexJoinIndexLookUp(c *C) {
func TestPartitionTableIndexJoinIndexLookUp(t *testing.T) {
if israce.RaceEnabled {
c.Skip("exhaustive types test, skip race test")
t.Skip("exhaustive types test, skip race test")
}
tk := testkit.NewTestKit(c, s.store)
t.Parallel()
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("set @@tidb_partition_prune_mode='dynamic'")
tk.MustExec(`create table t (a int, b int, key(a)) partition by hash(a) partitions 4`)
Expand Down

0 comments on commit c57e629

Please sign in to comment.