Skip to content

Commit

Permalink
executor/test: fix data race of executor unit tests (pingcap#8807)
Browse files Browse the repository at this point in the history
  • Loading branch information
eurekaka authored and yu34po committed Jan 2, 2019
1 parent 195bc96 commit 61be66b
Show file tree
Hide file tree
Showing 4 changed files with 130 additions and 95 deletions.
79 changes: 4 additions & 75 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2321,10 +2321,9 @@ func (s *testSuite) TestIssue4024(c *C) {
}

const (
checkRequestOff = 0
checkRequestPriority = 1
checkRequestSyncLog = 3
checkDDLAddIndexPriority = 4
checkRequestOff = iota
checkRequestSyncLog
checkDDLAddIndexPriority
)

type checkRequestClient struct {
Expand All @@ -2351,14 +2350,7 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req *
c.mu.RLock()
checkFlags := c.mu.checkFlags
c.mu.RUnlock()
if checkFlags == checkRequestPriority {
switch req.Type {
case tikvrpc.CmdCop:
if c.getCheckPriority() != req.Priority {
return nil, errors.New("fail to set priority")
}
}
} else if checkFlags == checkRequestSyncLog {
if checkFlags == checkRequestSyncLog {
switch req.Type {
case tikvrpc.CmdPrewrite, tikvrpc.CmdCommit:
c.mu.RLock()
Expand Down Expand Up @@ -2502,69 +2494,6 @@ func (s *testSuite1) TestAlterTableComment(c *C) {
result.Check(testkit.Rows("table t comment"))
}

func (s *testSuite1) TestCoprocessorPriority(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create table t (id int primary key)")
tk.MustExec("create table t1 (id int, v int, unique index i_id (id))")
defer tk.MustExec("drop table t")
defer tk.MustExec("drop table t1")
tk.MustExec("insert into t values (1)")

// Insert some data to make sure plan build IndexLookup for t1.
for i := 0; i < 10; i++ {
tk.MustExec(fmt.Sprintf("insert into t1 values (%d, %d)", i, i))
}

cli := s.cli
cli.mu.Lock()
cli.mu.checkFlags = checkRequestPriority
cli.mu.Unlock()

cli.setCheckPriority(pb.CommandPri_High)
tk.MustQuery("select id from t where id = 1")
tk.MustQuery("select * from t1 where id = 1")

cli.setCheckPriority(pb.CommandPri_Normal)
tk.MustQuery("select count(*) from t")
tk.MustExec("update t set id = 3")
tk.MustExec("delete from t")
tk.MustExec("insert into t select * from t limit 2")
tk.MustExec("delete from t")

// Insert some data to make sure plan build IndexLookup for t.
tk.MustExec("insert into t values (1), (2)")

oldThreshold := config.GetGlobalConfig().Log.ExpensiveThreshold
config.GetGlobalConfig().Log.ExpensiveThreshold = 0
defer func() { config.GetGlobalConfig().Log.ExpensiveThreshold = oldThreshold }()

cli.setCheckPriority(pb.CommandPri_High)
tk.MustQuery("select id from t where id = 1")
tk.MustQuery("select * from t1 where id = 1")

cli.setCheckPriority(pb.CommandPri_Low)
tk.MustQuery("select count(*) from t")
tk.MustExec("delete from t")
tk.MustExec("insert into t values (3)")

// TODO: Those are not point get, but they should be high priority.
// cli.priority = pb.CommandPri_High
// tk.MustExec("delete from t where id = 2")
// tk.MustExec("update t set id = 2 where id = 1")

// Test priority specified by SQL statement.
cli.setCheckPriority(pb.CommandPri_High)
tk.MustQuery("select HIGH_PRIORITY * from t")

cli.setCheckPriority(pb.CommandPri_Low)
tk.MustQuery("select LOW_PRIORITY id from t where id = 1")

cli.mu.Lock()
cli.mu.checkFlags = checkRequestOff
cli.mu.Unlock()
}

func (s *testSuite) TestTimezonePushDown(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
11 changes: 1 addition & 10 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"time"

. "github.com/pingcap/check"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/util/testkit"
)
Expand Down Expand Up @@ -118,14 +117,6 @@ func (s *testSuite2) TestJoin(c *C) {
result = tk.MustQuery("select a.c1 from t a , (select * from t1 limit 3) b where a.c1 = b.c1 order by b.c1;")
result.Check(testkit.Rows("1", "2", "3"))

plannercore.AllowCartesianProduct = false
err := tk.ExecToErr("select * from t, t1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select * from t left join t1 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select * from t right join t1 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
plannercore.AllowCartesianProduct = true
tk.MustExec("drop table if exists t,t2,t1")
tk.MustExec("create table t(c1 int)")
tk.MustExec("create table t1(c1 int, c2 int)")
Expand All @@ -152,7 +143,7 @@ func (s *testSuite2) TestJoin(c *C) {
tk.MustQuery("select /*+ TIDB_INLJ(t) */ avg(t.b) from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1.5000"))

// Test that two conflict hints will return error.
err = tk.ExecToErr("select /*+ TIDB_INLJ(t) TIDB_SMJ(t) */ * from t join t1 on t.a=t1.a")
err := tk.ExecToErr("select /*+ TIDB_INLJ(t) TIDB_SMJ(t) */ * from t join t1 on t.a=t1.a")
c.Assert(err, NotNil)
err = tk.ExecToErr("select /*+ TIDB_INLJ(t) TIDB_HJ(t) */ from t join t1 on t.a=t1.a")
c.Assert(err, NotNil)
Expand Down
11 changes: 1 addition & 10 deletions executor/merge_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"strings"

. "github.com/pingcap/check"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/util/testkit"
)

Expand Down Expand Up @@ -219,7 +218,7 @@ func checkMergeAndRun(tk *testkit.TestKit, c *C, sql string) *testkit.Result {
result := tk.MustQuery(explainedSQL)
resultStr := fmt.Sprintf("%v", result.Rows())
if !strings.ContainsAny(resultStr, "MergeJoin") {
c.Error("Expected MergeJoin in plannercore.")
c.Error("Expected MergeJoin in plan.")
}
return tk.MustQuery(sql)
}
Expand Down Expand Up @@ -302,14 +301,6 @@ func (s *testSuite1) TestMergeJoin(c *C) {
result.Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7"))
tk.MustExec("rollback;")

plannercore.AllowCartesianProduct = false
err := tk.ExecToErr("select /*+ TIDB_SMJ(t,t1) */ * from t, t1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select /*+ TIDB_SMJ(t,t1) */ * from t left join t1 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select /*+ TIDB_SMJ(t,t1) */ * from t right join t1 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
plannercore.AllowCartesianProduct = true
tk.MustExec("drop table if exists t")
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t(c1 int)")
Expand Down
124 changes: 124 additions & 0 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,19 +28,24 @@ import (
"time"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
gofail "github.com/pingcap/gofail/runtime"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/parser"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -716,3 +721,122 @@ func generateBatchSQL(paramCount int) (sql string, paramSlice []interface{}) {
}
return "insert into t values " + strings.Join(placeholders, ","), params
}

func (s *seqTestSuite) TestCartesianProduct(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(c1 int)")
plannercore.AllowCartesianProduct = false
err := tk.ExecToErr("select * from t t1, t t2")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select * from t t1 left join t t2 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select * from t t1 right join t t2 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
plannercore.AllowCartesianProduct = true
}

type checkPrioClient struct {
tikv.Client
priority pb.CommandPri
}

func (c *checkPrioClient) setCheckPriority(priority pb.CommandPri) {
atomic.StoreInt32((*int32)(&c.priority), int32(priority))
}

func (c *checkPrioClient) getCheckPriority() pb.CommandPri {
return (pb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority)))
}

func (c *checkPrioClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) {
resp, err := c.Client.SendRequest(ctx, addr, req, timeout)
switch req.Type {
case tikvrpc.CmdCop:
if c.getCheckPriority() != req.Priority {
return nil, errors.New("fail to set priority")
}
}
return resp, err
}

type seqTestSuite1 struct {
store kv.Storage
dom *domain.Domain
cli *checkPrioClient
}

func (s *seqTestSuite1) SetUpSuite(c *C) {
cli := &checkPrioClient{}
hijackClient := func(c tikv.Client) tikv.Client {
cli.Client = c
return cli
}
s.cli = cli

var err error
s.store, err = mockstore.NewMockTikvStore(
mockstore.WithHijackClient(hijackClient),
)
c.Assert(err, IsNil)
s.dom, err = session.BootstrapSession(s.store)
c.Assert(err, IsNil)
}

func (s *seqTestSuite1) TearDownSuite(c *C) {
s.dom.Close()
s.store.Close()
}

func (s *seqTestSuite1) TestCoprocessorPriority(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create table t (id int primary key)")
tk.MustExec("create table t1 (id int, v int, unique index i_id (id))")
defer tk.MustExec("drop table t")
defer tk.MustExec("drop table t1")
tk.MustExec("insert into t values (1)")

// Insert some data to make sure plan build IndexLookup for t1.
for i := 0; i < 10; i++ {
tk.MustExec(fmt.Sprintf("insert into t1 values (%d, %d)", i, i))
}

cli := s.cli
cli.setCheckPriority(pb.CommandPri_High)
tk.MustQuery("select id from t where id = 1")
tk.MustQuery("select * from t1 where id = 1")

cli.setCheckPriority(pb.CommandPri_Normal)
tk.MustQuery("select count(*) from t")
tk.MustExec("update t set id = 3")
tk.MustExec("delete from t")
tk.MustExec("insert into t select * from t limit 2")
tk.MustExec("delete from t")

// Insert some data to make sure plan build IndexLookup for t.
tk.MustExec("insert into t values (1), (2)")

oldThreshold := config.GetGlobalConfig().Log.ExpensiveThreshold
config.GetGlobalConfig().Log.ExpensiveThreshold = 0
defer func() { config.GetGlobalConfig().Log.ExpensiveThreshold = oldThreshold }()

cli.setCheckPriority(pb.CommandPri_High)
tk.MustQuery("select id from t where id = 1")
tk.MustQuery("select * from t1 where id = 1")
tk.MustExec("delete from t where id = 2")
tk.MustExec("update t set id = 2 where id = 1")

cli.setCheckPriority(pb.CommandPri_Low)
tk.MustQuery("select count(*) from t")
tk.MustExec("delete from t")
tk.MustExec("insert into t values (3)")

// Test priority specified by SQL statement.
cli.setCheckPriority(pb.CommandPri_High)
tk.MustQuery("select HIGH_PRIORITY * from t")

cli.setCheckPriority(pb.CommandPri_Low)
tk.MustQuery("select LOW_PRIORITY id from t where id = 1")
}

0 comments on commit 61be66b

Please sign in to comment.