diff --git a/Makefile b/Makefile index 6c6117fb1c2fb..ee4113c963857 100644 --- a/Makefile +++ b/Makefile @@ -14,7 +14,7 @@ export PATH := $(path_to_add):$(PATH) GO := GO111MODULE=on go GOBUILD := $(GO) build $(BUILD_FLAG) -tags codes GOBUILDCOVERAGE := GOPATH=$(GOPATH) cd tidb-server; $(GO) test -coverpkg="../..." -c . -GOTEST := $(GO) test -p 4 +GOTEST := $(GO) test -p 8 OVERALLS := GO111MODULE=on overalls ARCH := "`uname -s`" diff --git a/config/config.go b/config/config.go index ddf6a37937b11..2b6d494e94a99 100644 --- a/config/config.go +++ b/config/config.go @@ -404,7 +404,7 @@ var defaultConf = Config{ Reporter: OpenTracingReporter{}, }, TiKVClient: TiKVClient{ - GrpcConnectionCount: 16, + GrpcConnectionCount: 4, GrpcKeepAliveTime: 10, GrpcKeepAliveTimeout: 3, CommitTimeout: "41s", diff --git a/config/config.toml.example b/config/config.toml.example index e98b19e14bcab..6c38768df569f 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -259,7 +259,7 @@ local-agent-host-port = "" [tikv-client] # Max gRPC connections that will be established with each tikv-server. -grpc-connection-count = 16 +grpc-connection-count = 4 # After a duration of this time in seconds if the client doesn't see any activity it pings # the server to see if the transport is still alive. diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 2910f11e2bac3..6362082d0a58d 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -69,6 +69,11 @@ func setupIntegrationSuite(s *testIntegrationSuite, c *C) { s.lease = 50 * time.Millisecond ddl.WaitTimeWhenErrorOccured = 0 + cfg := config.GetGlobalConfig() + newCfg := *cfg + newCfg.Log.SlowThreshold = 10000 + config.StoreGlobalConfig(&newCfg) + s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) s.mvccStore = mocktikv.MustNewMVCCStore() diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 95bf83f851033..e2c2971f32d9b 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -1248,9 +1248,7 @@ func (s *testIntegrationSuite2) TestPartitionCancelAddIndex(c *C) { base := defaultBatchSize * 2 count := base // add some rows - for i := 0; i < count; i++ { - tk.MustExec("insert into t1 values (?, ?, ?)", i, i, i) - } + batchInsert(s.tk, "t1", 0, count) var checkErr error var c3IdxInfo *model.IndexInfo diff --git a/ddl/db_test.go b/ddl/db_test.go index bc9197a5ac14e..056b28e616384 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -94,6 +94,7 @@ func setUpSuite(s *testDBSuite, c *C) { cfg := config.GetGlobalConfig() newCfg := *cfg newCfg.EnableTableLock = true + newCfg.Log.SlowThreshold = 10000 config.StoreGlobalConfig(&newCfg) s.cluster = mocktikv.NewCluster() @@ -225,6 +226,17 @@ func backgroundExec(s kv.Storage, sql string, done chan error) { done <- errors.Trace(err) } +func batchInsert(tk *testkit.TestKit, tbl string, start, end int) { + dml := fmt.Sprintf("insert into %s values", tbl) + for i := start; i < end; i++ { + dml += fmt.Sprintf("(%d, %d, %d)", i, i, i) + if i != end-1 { + dml += "," + } + } + tk.MustExec(dml) +} + func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") @@ -234,9 +246,7 @@ func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) { base := defaultBatchSize * 2 count := base // add some rows - for i := 0; i < count; i++ { - s.mustExec(c, "insert into t1 values (?, ?, ?)", i, i, i) - } + batchInsert(s.tk, "t1", 0, count) // add some duplicate rows for i := count - 10; i < count; i++ { s.mustExec(c, "insert into t1 values (?, ?, ?)", i+10, i, i) @@ -1330,9 +1340,7 @@ func (s *testDBSuite) testAddColumn(c *C) { num := defaultBatchSize + 10 // add some rows - for i := 0; i < num; i++ { - s.mustExec(c, "insert into t2 values (?, ?, ?)", i, i, i) - } + batchInsert(s.tk, "t2", 0, num) testddlutil.SessionExecInGoroutine(c, s.store, "alter table t2 add column c4 int default -1", done) @@ -1519,15 +1527,22 @@ func (s *testDBSuite2) TestDropColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("create database drop_col_db") s.tk.MustExec("use drop_col_db") - s.tk.MustExec("create table t2 (c1 int, c2 int, c3 int)") - num := 50 + num := 25 + multiDDL := make([]string, 0, num) + sql := "create table t2 (c1 int, c2 int, c3 int, " + for i := 4; i < 4+num; i++ { + multiDDL = append(multiDDL, fmt.Sprintf("alter table t2 drop column c%d", i)) + + if i != 3+num { + sql += fmt.Sprintf("c%d int, ", i) + } else { + sql += fmt.Sprintf("c%d int)", i) + } + } + s.tk.MustExec(sql) dmlDone := make(chan error, num) ddlDone := make(chan error, num) - multiDDL := make([]string, 0, num) - for i := 0; i < num/2; i++ { - multiDDL = append(multiDDL, "alter table t2 add column c4 int", "alter table t2 drop column c4") - } testddlutil.ExecMultiSQLInGoroutine(c, s.store, "drop_col_db", multiDDL, ddlDone) for i := 0; i < num; i++ { testddlutil.ExecMultiSQLInGoroutine(c, s.store, "drop_col_db", []string{"insert into t2 set c1 = 1, c2 = 1, c3 = 1, c4 = 1"}, dmlDone) @@ -2465,16 +2480,17 @@ func (s *testDBSuite2) TestAddNotNullColumnWhileInsertOnDupUpdate(c *C) { return default: } - _, tk2Err = tk2.Exec("insert nn (a, b) values (1, 1) on duplicate key update a = 1, b = b + 1") + _, tk2Err = tk2.Exec("insert nn (a, b) values (1, 1) on duplicate key update a = 1, b = values(b) + 1") if tk2Err != nil { return } } }() - tk1.MustExec("alter table nn add column c int not null default 0") + tk1.MustExec("alter table nn add column c int not null default 3 after a") close(closeCh) wg.Wait() c.Assert(tk2Err, IsNil) + tk1.MustQuery("select * from nn").Check(testkit.Rows("1 3 2")) } func (s *testDBSuite3) TestColumnModifyingDefinition(c *C) { diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 27285c52b041a..fbf4ee15f784a 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -325,6 +325,17 @@ func (s *testFailDBSuite) TestGenGlobalIDFail(c *C) { tk.MustExec("admin check table t2") } +func batchInsert(tk *testkit.TestKit, tbl string, start, end int) { + dml := fmt.Sprintf("insert into %s values", tbl) + for i := start; i < end; i++ { + dml += fmt.Sprintf("(%d, %d, %d)", i, i, i) + if i != end-1 { + dml += "," + } + } + tk.MustExec(dml) +} + func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_db") @@ -334,11 +345,9 @@ func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) { done := make(chan error, 1) start := -10 - num := 4096 // first add some rows - for i := start; i < num; i++ { - sql := fmt.Sprintf("insert into test_add_index values (%d, %d, %d)", i, i, i) - tk.MustExec(sql) + for i := start; i < 4090; i += 100 { + batchInsert(tk, "test_add_index", i, i+100) } is := s.dom.InfoSchema() diff --git a/ddl/index.go b/ddl/index.go index 1716cf2df3b4b..ebcffe21ea916 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -869,7 +869,7 @@ func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (taskCtx // Lock the row key to notify us that someone delete or update the row, // then we should not backfill the index of it, otherwise the adding index is redundant. - err := txn.LockKeys(context.Background(), 0, idxRecord.key) + err := txn.LockKeys(context.Background(), nil, 0, idxRecord.key) if err != nil { return errors.Trace(err) } diff --git a/domain/domain.go b/domain/domain.go index 3e829c7511114..0d5aeb11ce813 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -421,7 +421,7 @@ func (do *Domain) topNSlowQueryLoop() { func (do *Domain) infoSyncerKeeper() { defer do.wg.Done() defer recoverInDomain("infoSyncerKeeper", false) - ticker := time.NewTicker(time.Second * time.Duration(infosync.InfoSessionTTL) / 2) + ticker := time.NewTicker(infosync.ReportInterval) defer ticker.Stop() for { select { diff --git a/domain/infosync/info.go b/domain/infosync/info.go index 244592f08dc91..e725ee16522ec 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -43,14 +43,15 @@ const ( // ServerMinStartTSPath store the server min start timestamp. ServerMinStartTSPath = "/tidb/server/minstartts" // keyOpDefaultRetryCnt is the default retry count for etcd store. - keyOpDefaultRetryCnt = 2 + keyOpDefaultRetryCnt = 5 // keyOpDefaultTimeout is the default time out for etcd store. keyOpDefaultTimeout = 1 * time.Second + // InfoSessionTTL is the ETCD session's TTL in seconds. + InfoSessionTTL = 10 * 60 + // ReportInterval is interval of infoSyncerKeeper reporting min startTS. + ReportInterval = 30 * time.Second ) -// InfoSessionTTL is the etcd session's TTL in seconds. It's exported for testing. -var InfoSessionTTL = 1 * 60 - // InfoSyncer stores server info to etcd when the tidb-server starts and delete when tidb-server shuts down. type InfoSyncer struct { etcdCli *clientv3.Client diff --git a/executor/adapter.go b/executor/adapter.go index c81cd77341a93..1dc6015a7f563 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -557,7 +557,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { return nil } forUpdateTS := txnCtx.GetForUpdateTS() - err = txn.LockKeys(ctx, forUpdateTS, keys...) + err = txn.LockKeys(ctx, &sctx.GetSessionVars().Killed, forUpdateTS, keys...) if err == nil { return nil } diff --git a/executor/adapter_test.go b/executor/adapter_test.go index e4ae20c9b8c60..bf33a3bf4f1be 100644 --- a/executor/adapter_test.go +++ b/executor/adapter_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuiteP1) TestQueryTime(c *C) { +func (s *testSuiteP2) TestQueryTime(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/admin.go b/executor/admin.go index c9d0aa5f7fb6c..dde2395a23518 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -432,7 +432,7 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa } recordKey := e.table.RecordKey(row.handle) - err := txn.LockKeys(ctx, 0, recordKey) + err := txn.LockKeys(ctx, nil, 0, recordKey) if err != nil { return result, err } diff --git a/executor/admin_test.go b/executor/admin_test.go index 4d59ebb926b81..a914c2c00836a 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -44,7 +44,7 @@ func (s *testSuite1) TestAdminCheckIndexRange(c *C) { result.Check(testkit.Rows("-1 hi 4", "2 cd 2")) } -func (s *testSuite2) TestAdminRecoverIndex(c *C) { +func (s *testSuite5) TestAdminRecoverIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") @@ -158,7 +158,7 @@ func (s *testSuite2) TestAdminRecoverIndex(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite2) TestAdminRecoverIndex1(c *C) { +func (s *testSuite5) TestAdminRecoverIndex1(c *C) { tk := testkit.NewTestKit(c, s.store) s.ctx = mock.NewContext() s.ctx.Store = s.store @@ -209,7 +209,7 @@ func (s *testSuite2) TestAdminRecoverIndex1(c *C) { tk.MustExec("admin check index admin_test `primary`") } -func (s *testSuite2) TestAdminCleanupIndex(c *C) { +func (s *testSuite5) TestAdminCleanupIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") @@ -288,7 +288,7 @@ func (s *testSuite2) TestAdminCleanupIndex(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite2) TestAdminCleanupIndexPKNotHandle(c *C) { +func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") @@ -336,7 +336,7 @@ func (s *testSuite2) TestAdminCleanupIndexPKNotHandle(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite2) TestAdminCleanupIndexMore(c *C) { +func (s *testSuite5) TestAdminCleanupIndexMore(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") @@ -399,7 +399,7 @@ func (s *testSuite2) TestAdminCleanupIndexMore(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite2) TestAdminCheckTableFailed(c *C) { +func (s *testSuite5) TestAdminCheckTableFailed(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") @@ -505,7 +505,7 @@ func (s *testSuite2) TestAdminCheckTableFailed(c *C) { tk.MustExec("admin check table admin_test") } -func (s *testSuite1) TestAdminCheckTable(c *C) { +func (s *testSuite2) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -631,7 +631,7 @@ func (s *testSuite1) TestAdminCheckPrimaryIndex(c *C) { tk.MustExec("admin check index t idx;") } -func (s *testSuite2) TestAdminCheckWithSnapshot(c *C) { +func (s *testSuite5) TestAdminCheckWithSnapshot(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_t_s") diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index a6d6a61cd4841..72e36fe846281 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -21,7 +21,11 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite1) TestAggregation(c *C) { +type testSuiteAgg struct { + *baseTestSuite +} + +func (s *testSuiteAgg) TestAggregation(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("use test") @@ -353,7 +357,7 @@ func (s *testSuite1) TestAggregation(c *C) { c.Assert(errors.Cause(err).Error(), Equals, "unsupported agg function: var_samp") } -func (s *testSuite1) TestAggPrune(c *C) { +func (s *testSuiteAgg) TestAggPrune(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -377,7 +381,7 @@ func (s *testSuite1) TestAggPrune(c *C) { tk.MustQuery("SELECT a, MIN(b), MAX(b) FROM t GROUP BY a").Check(testkit.Rows("1 11 11", "3 ")) } -func (s *testSuite1) TestGroupConcatAggr(c *C) { +func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { // issue #5411 tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -410,7 +414,7 @@ func (s *testSuite1) TestGroupConcatAggr(c *C) { tk.MustQuery("select group_concat(123, null)").Check(testkit.Rows("")) } -func (s *testSuite) TestSelectDistinct(c *C) { +func (s *testSuiteAgg) TestSelectDistinct(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") s.fillData(tk, "select_distinct_test") @@ -422,7 +426,7 @@ func (s *testSuite) TestSelectDistinct(c *C) { } -func (s *testSuite1) TestAggPushDown(c *C) { +func (s *testSuiteAgg) TestAggPushDown(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -450,7 +454,7 @@ func (s *testSuite1) TestAggPushDown(c *C) { tk.MustQuery("select a, count(b) from (select * from t union all select * from tt) k group by a order by a").Check(testkit.Rows("1 2", "2 1")) } -func (s *testSuite1) TestOnlyFullGroupBy(c *C) { +func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") @@ -562,7 +566,7 @@ func (s *testSuite1) TestOnlyFullGroupBy(c *C) { c.Assert(terror.ErrorEqual(err, plannercore.ErrAmbiguous), IsTrue, Commentf("err %v", err)) } -func (s *testSuite1) TestHaving(c *C) { +func (s *testSuiteAgg) TestHaving(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("set sql_mode = 'STRICT_TRANS_TABLES,NO_ENGINE_SUBSTITUTION'") @@ -587,7 +591,7 @@ func (s *testSuite1) TestHaving(c *C) { tk.MustQuery("select 1 from t group by c1 having sum(abs(c2 + c3)) = c1").Check(testkit.Rows("1")) } -func (s *testSuite1) TestAggEliminator(c *C) { +func (s *testSuiteAgg) TestAggEliminator(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int primary key, b int)") @@ -600,7 +604,7 @@ func (s *testSuite1) TestAggEliminator(c *C) { tk.MustQuery("select group_concat(b, b) from t group by a").Sort().Check(testkit.Rows("-1-1", "-2-2", "11", "")) } -func (s *testSuite1) TestMaxMinFloatScalaFunc(c *C) { +func (s *testSuiteAgg) TestMaxMinFloatScalaFunc(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec(`DROP TABLE IF EXISTS T;`) @@ -610,7 +614,7 @@ func (s *testSuite1) TestMaxMinFloatScalaFunc(c *C) { tk.MustQuery(`SELECT MIN(CASE B WHEN 'val_b' THEN C ELSE 0 END) val_b FROM T WHERE cast(A as signed) = 0 GROUP BY a;`).Check(testkit.Rows("12.190999984741211")) } -func (s *testSuite1) TestBuildProjBelowAgg(c *C) { +func (s *testSuiteAgg) TestBuildProjBelowAgg(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t;") tk.MustExec("create table t (i int);") @@ -622,7 +626,7 @@ func (s *testSuite1) TestBuildProjBelowAgg(c *C) { "4 3 18 7,7,7 8")) } -func (s *testSuite1) TestInjectProjBelowTopN(c *C) { +func (s *testSuiteAgg) TestInjectProjBelowTopN(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t;") tk.MustExec("create table t (i int);") @@ -648,7 +652,7 @@ func (s *testSuite1) TestInjectProjBelowTopN(c *C) { tk.MustQuery("select i, i, i from t order by i + 1").Check(testkit.Rows("1 1 1", "1 1 1", "1 1 1", "2 2 2", "2 2 2", "2 2 2", "3 3 3", "3 3 3", "3 3 3")) } -func (s *testSuite1) TestFirstRowEnum(c *C) { +func (s *testSuiteAgg) TestFirstRowEnum(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec(`use test;`) tk.MustExec(`drop table if exists t;`) @@ -659,7 +663,7 @@ func (s *testSuite1) TestFirstRowEnum(c *C) { )) } -func (s *testSuite1) TestAggJSON(c *C) { +func (s *testSuiteAgg) TestAggJSON(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a datetime, b json, index idx(a));`) @@ -708,7 +712,7 @@ func (s *testSuite1) TestAggJSON(c *C) { )) } -func (s *testSuite1) TestIssue10099(c *C) { +func (s *testSuiteAgg) TestIssue10099(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") tk.MustExec("create table t(a char(10), b char(10))") @@ -716,7 +720,7 @@ func (s *testSuite1) TestIssue10099(c *C) { tk.MustQuery("select count(distinct a, b) from t").Check(testkit.Rows("2")) } -func (s *testSuite1) TestIssue10098(c *C) { +func (s *testSuiteAgg) TestIssue10098(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec(`drop table if exists t;`) tk.MustExec("create table t(a char(10), b char(10))") @@ -724,7 +728,7 @@ func (s *testSuite1) TestIssue10098(c *C) { tk.MustQuery("select group_concat(distinct a, b) from t").Check(testkit.Rows("1222,1222")) } -func (s *testSuite1) TestIssue10608(c *C) { +func (s *testSuiteAgg) TestIssue10608(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec(`drop table if exists t, s;`) tk.MustExec("create table t(a int)") @@ -736,7 +740,7 @@ func (s *testSuite1) TestIssue10608(c *C) { } -func (s *testSuite1) TestIssue12759HashAggCalledByApply(c *C) { +func (s *testSuiteAgg) TestIssue12759HashAggCalledByApply(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.Se.GetSessionVars().HashAggFinalConcurrency = 4 tk.MustExec(`insert into mysql.opt_rule_blacklist value("decorrelate");`) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 0f68cf20017d4..1520d0b9126eb 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -251,7 +251,7 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { c.Assert(fmt.Sprintln(vals), Equals, "[[0 4 6 9 10 11 12 14 17 24 25 29 30 34 35 44 52 54 57 58] [0 4 6 9 10 11 12 14 17 24 25 29 30 34 35 44 52 54 57 58]]\n") } -func (s *testSuite1) TestFastAnalyze(c *C) { +func (s *testFastAnalyze) TestFastAnalyze(c *C) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) store, err := mockstore.NewMockTikvStore( diff --git a/executor/builder.go b/executor/builder.go index 9b8465268121a..6651da0c90bf7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -865,8 +865,9 @@ func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannerco // GetDirtyDB() is safe here. If this table has been modified in the transaction, non-nil DirtyTable // can be found in DirtyDB now, so GetDirtyTable is safe; if this table has not been modified in the // transaction, empty DirtyTable would be inserted into DirtyDB, it does not matter when multiple - // goroutines write empty DirtyTable to DirtyDB for this table concurrently. Thus we don't use lock - // to synchronize here. + // goroutines write empty DirtyTable to DirtyDB for this table concurrently. Although the DirtyDB looks + // safe for data race in all the cases, the map of golang will throw panic when it's accessed in parallel. + // So we lock it when getting dirty table. physicalTableID := getPhysicalTableID(x.table) us.dirty = GetDirtyDB(b.ctx).GetDirtyTable(physicalTableID) us.conditions = v.Conditions @@ -2230,7 +2231,7 @@ func (builder *dataReaderBuilder) buildProjectionForIndexJoin(ctx context.Contex if int64(v.StatsCount()) < int64(builder.ctx.GetSessionVars().MaxChunkSize) { e.numWorkers = 0 } - err = e.Open(ctx) + err = e.open(ctx) return e, err } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 66c9c607bca9d..3e009fed443aa 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/ddl" ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -42,7 +43,7 @@ import ( "github.com/pingcap/tidb/util/testutil" ) -func (s *testSuite3) TestTruncateTable(c *C) { +func (s *testSuite6) TestTruncateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`drop table if exists truncate_test;`) @@ -59,7 +60,7 @@ func (s *testSuite3) TestTruncateTable(c *C) { // 1. Execute the SQL of "begin"; // 2. A SQL that will fail to execute; // 3. Execute DDL. -func (s *testSuite3) TestInTxnExecDDLFail(c *C) { +func (s *testSuite6) TestInTxnExecDDLFail(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t (i int key);") @@ -72,7 +73,7 @@ func (s *testSuite3) TestInTxnExecDDLFail(c *C) { result.Check(testkit.Rows("1")) } -func (s *testSuite3) TestCreateTable(c *C) { +func (s *testSuite6) TestCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // Test create an exist database @@ -173,7 +174,7 @@ func (s *testSuite3) TestCreateTable(c *C) { r.Check(testkit.Rows("1000 aa")) } -func (s *testSuite3) TestCreateView(c *C) { +func (s *testSuite6) TestCreateView(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") //create an source table @@ -224,9 +225,36 @@ func (s *testSuite3) TestCreateView(c *C) { // create view using prepare tk.MustExec(`prepare stmt from "create view v10 (x) as select 1";`) tk.MustExec("execute stmt") + + // create view on union + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("drop view if exists v") + _, err = tk.Exec("create view v as select * from t1 union select * from t2") + c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotExists), IsTrue) + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("create table t2(a int, b int)") + tk.MustExec("insert into t1 values(1,2), (1,1), (1,2)") + tk.MustExec("insert into t2 values(1,1),(1,3)") + tk.MustExec("create definer='root'@'localhost' view v as select * from t1 union select * from t2") + tk.MustQuery("select * from v").Sort().Check(testkit.Rows("1 1", "1 2", "1 3")) + tk.MustExec("alter table t1 drop column a") + _, err = tk.Exec("select * from v") + c.Assert(terror.ErrorEqual(err, plannercore.ErrViewInvalid), IsTrue) + tk.MustExec("alter table t1 add column a int") + tk.MustQuery("select * from v").Sort().Check(testkit.Rows("1 1", "1 3", " 1", " 2")) + tk.MustExec("alter table t1 drop column a") + tk.MustExec("alter table t2 drop column b") + _, err = tk.Exec("select * from v") + c.Assert(terror.ErrorEqual(err, plannercore.ErrViewInvalid), IsTrue) + tk.MustExec("drop view v") + + tk.MustExec("create view v as (select * from t1)") + tk.MustExec("drop view v") + tk.MustExec("create view v as (select * from t1 union select * from t2)") + tk.MustExec("drop view v") } -func (s *testSuite3) TestCreateDropDatabase(c *C) { +func (s *testSuite6) TestCreateDropDatabase(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists drop_test;") tk.MustExec("drop database if exists drop_test;") @@ -242,7 +270,7 @@ func (s *testSuite3) TestCreateDropDatabase(c *C) { c.Assert(err, NotNil) } -func (s *testSuite3) TestCreateDropTable(c *C) { +func (s *testSuite6) TestCreateDropTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table if not exists drop_test (a int)") @@ -254,7 +282,7 @@ func (s *testSuite3) TestCreateDropTable(c *C) { c.Assert(err, NotNil) } -func (s *testSuite3) TestCreateDropView(c *C) { +func (s *testSuite6) TestCreateDropView(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create or replace view drop_test as select 1,2") @@ -272,7 +300,7 @@ func (s *testSuite3) TestCreateDropView(c *C) { c.Assert(err.Error(), Equals, "[ddl:1347]'test.t_v' is not VIEW") } -func (s *testSuite3) TestCreateDropIndex(c *C) { +func (s *testSuite6) TestCreateDropIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table if not exists drop_test (a int)") @@ -281,7 +309,7 @@ func (s *testSuite3) TestCreateDropIndex(c *C) { tk.MustExec("drop table drop_test") } -func (s *testSuite3) TestAlterTableAddColumn(c *C) { +func (s *testSuite6) TestAlterTableAddColumn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table if not exists alter_test (c1 int)") @@ -306,7 +334,7 @@ func (s *testSuite3) TestAlterTableAddColumn(c *C) { tk.MustExec("drop view alter_view") } -func (s *testSuite3) TestAddNotNullColumnNoDefault(c *C) { +func (s *testSuite6) TestAddNotNullColumnNoDefault(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table nn (c1 int)") @@ -327,7 +355,7 @@ func (s *testSuite3) TestAddNotNullColumnNoDefault(c *C) { tk.MustQuery("select * from nn").Check(testkit.Rows("1 0", "2 0", "3 0")) } -func (s *testSuite3) TestAlterTableModifyColumn(c *C) { +func (s *testSuite6) TestAlterTableModifyColumn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists mc") @@ -392,7 +420,7 @@ func (s *testSuite3) TestAlterTableModifyColumn(c *C) { } -func (s *testSuite3) TestDefaultDBAfterDropCurDB(c *C) { +func (s *testSuite6) TestDefaultDBAfterDropCurDB(c *C) { tk := testkit.NewTestKit(c, s.store) testSQL := `create database if not exists test_db CHARACTER SET latin1 COLLATE latin1_swedish_ci;` @@ -411,7 +439,7 @@ func (s *testSuite3) TestDefaultDBAfterDropCurDB(c *C) { tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows("utf8_unicode_ci")) } -func (s *testSuite3) TestRenameTable(c *C) { +func (s *testSuite6) TestRenameTable(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) @@ -471,7 +499,7 @@ func (s *testSuite3) TestRenameTable(c *C) { tk.MustExec("drop database rename2") } -func (s *testSuite3) TestColumnCharsetAndCollate(c *C) { +func (s *testSuite6) TestColumnCharsetAndCollate(c *C) { tk := testkit.NewTestKit(c, s.store) dbName := "col_charset_collate" tk.MustExec("create database " + dbName) @@ -555,7 +583,7 @@ func (s *testSuite3) TestColumnCharsetAndCollate(c *C) { tk.MustExec("drop database " + dbName) } -func (s *testSuite3) TestTooLargeIdentifierLength(c *C) { +func (s *testSuite6) TestTooLargeIdentifierLength(c *C) { tk := testkit.NewTestKit(c, s.store) // for database. @@ -590,7 +618,7 @@ func (s *testSuite3) TestTooLargeIdentifierLength(c *C) { c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", indexName2)) } -func (s *testSuite3) TestShardRowIDBits(c *C) { +func (s *testSuite8) TestShardRowIDBits(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -703,7 +731,7 @@ func (s *testSuite3) TestShardRowIDBits(c *C) { c.Assert(autoid.ErrAutoincReadFailed.Equal(err), IsTrue, Commentf("err:%v", err)) } -func (s *testSuite3) TestMaxHandleAddIndex(c *C) { +func (s *testSuite6) TestMaxHandleAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -720,7 +748,7 @@ func (s *testSuite3) TestMaxHandleAddIndex(c *C) { tk.MustExec("admin check table t1") } -func (s *testSuite3) TestSetDDLReorgWorkerCnt(c *C) { +func (s *testSuite6) TestSetDDLReorgWorkerCnt(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") err := ddlutil.LoadDDLReorgVars(tk.Se) @@ -754,7 +782,7 @@ func (s *testSuite3) TestSetDDLReorgWorkerCnt(c *C) { res.Check(testkit.Rows("100")) } -func (s *testSuite3) TestSetDDLReorgBatchSize(c *C) { +func (s *testSuite6) TestSetDDLReorgBatchSize(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") err := ddlutil.LoadDDLReorgVars(tk.Se) @@ -791,7 +819,7 @@ func (s *testSuite3) TestSetDDLReorgBatchSize(c *C) { res.Check(testkit.Rows("1000")) } -func (s *testSuite3) TestIllegalFunctionCall4GeneratedColumns(c *C) { +func (s *testSuite6) TestIllegalFunctionCall4GeneratedColumns(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // Test create an exist database @@ -836,7 +864,7 @@ func (s *testSuite3) TestIllegalFunctionCall4GeneratedColumns(c *C) { c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("cc").Error()) } -func (s *testSuite3) TestGeneratedColumnRelatedDDL(c *C) { +func (s *testSuite6) TestGeneratedColumnRelatedDDL(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // Test create an exist database @@ -862,7 +890,7 @@ func (s *testSuite3) TestGeneratedColumnRelatedDDL(c *C) { tk.MustExec("drop table t1;") } -func (s *testSuite3) TestSetDDLErrorCountLimit(c *C) { +func (s *testSuite6) TestSetDDLErrorCountLimit(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") err := ddlutil.LoadDDLVars(tk.Se) @@ -891,7 +919,7 @@ func (s *testSuite3) TestSetDDLErrorCountLimit(c *C) { // Test issue #9205, fix the precision problem for time type default values // See https://github.com/pingcap/tidb/issues/9205 for details -func (s *testSuite3) TestIssue9205(c *C) { +func (s *testSuite6) TestIssue9205(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) @@ -930,7 +958,7 @@ func (s *testSuite3) TestIssue9205(c *C) { )) } -func (s *testSuite3) TestCheckDefaultFsp(c *C) { +func (s *testSuite6) TestCheckDefaultFsp(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) @@ -970,7 +998,7 @@ func (s *testSuite3) TestCheckDefaultFsp(c *C) { c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tttt'") } -func (s *testSuite3) TestTimestampMinDefaultValue(c *C) { +func (s *testSuite6) TestTimestampMinDefaultValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tdv;") diff --git a/executor/executor.go b/executor/executor.go index 167a187a8db18..95414231b8643 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -804,7 +804,7 @@ func doLockKeys(ctx context.Context, se sessionctx.Context, keys ...kv.Key) erro return err } forUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() - return txn.LockKeys(ctx, forUpdateTS, keys...) + return txn.LockKeys(ctx, &se.GetSessionVars().Killed, forUpdateTS, keys...) } // LimitExec represents limit executor diff --git a/executor/executor_test.go b/executor/executor_test.go index 0ca6d6a4e51d2..0bdca4d6acd4d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -83,6 +83,12 @@ func TestT(t *testing.T) { logLevel := os.Getenv("log_level") logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) autoid.SetStep(5000) + + old := config.GetGlobalConfig() + new := *old + new.Log.SlowThreshold = 30000 // 30s + config.StoreGlobalConfig(&new) + testleak.BeforeTest() TestingT(t) testleak.AfterTestT(t)() @@ -90,10 +96,19 @@ func TestT(t *testing.T) { var _ = Suite(&testSuite{&baseTestSuite{}}) var _ = Suite(&testSuiteP1{&baseTestSuite{}}) +var _ = Suite(&testSuiteP2{&baseTestSuite{}}) var _ = Suite(&testSuite1{}) -var _ = Suite(&testSuite2{}) -var _ = Suite(&testSuite3{}) -var _ = Suite(&testSuite4{}) +var _ = Suite(&testSuite2{&baseTestSuite{}}) +var _ = Suite(&testSuite3{&baseTestSuite{}}) +var _ = Suite(&testSuite4{&baseTestSuite{}}) +var _ = Suite(&testSuite5{&baseTestSuite{}}) +var _ = Suite(&testSuiteJoin1{&baseTestSuite{}}) +var _ = Suite(&testSuiteJoin2{&baseTestSuite{}}) +var _ = Suite(&testSuiteJoin3{&baseTestSuite{}}) +var _ = Suite(&testSuiteAgg{&baseTestSuite{}}) +var _ = Suite(&testSuite6{&baseTestSuite{}}) +var _ = Suite(&testSuite7{&baseTestSuite{}}) +var _ = Suite(&testSuite8{&baseTestSuite{}}) var _ = SerialSuites(&testShowStatsSuite{&baseTestSuite{}}) var _ = Suite(&testBypassSuite{}) var _ = Suite(&testUpdateSuite{}) @@ -105,6 +120,7 @@ var _ = Suite(&testFlushSuite{}) type testSuite struct{ *baseTestSuite } type testSuiteP1 struct{ *baseTestSuite } +type testSuiteP2 struct{ *baseTestSuite } type baseTestSuite struct { cluster *mocktikv.Cluster @@ -428,7 +444,7 @@ func (s *testSuiteP1) TestAdmin(c *C) { c.Assert(historyJobs, DeepEquals, historyJobs2) } -func (s *testSuite) TestAdminShowDDLJobs(c *C) { +func (s *testSuiteP2) TestAdminShowDDLJobs(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_admin_show_ddl_jobs") tk.MustExec("use test_admin_show_ddl_jobs") @@ -465,7 +481,7 @@ func (s *testSuite) TestAdminShowDDLJobs(c *C) { c.Assert(row[1], Equals, "test_admin_show_ddl_jobs") } -func (s *testSuite) TestAdminChecksumOfPartitionedTable(c *C) { +func (s *testSuiteP2) TestAdminChecksumOfPartitionedTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS admin_checksum_partition_test;") @@ -1066,7 +1082,7 @@ func (s *testSuiteP1) TestIssue5055(c *C) { result.Check(testkit.Rows("1 1")) } -func (s *testSuite) TestUnion(c *C) { +func (s *testSuiteP2) TestUnion(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1402,7 +1418,7 @@ func (s *testSuiteP1) TestTablePKisHandleScan(c *C) { } } -func (s *testSuiteP1) TestIndexScan(c *C) { +func (s *testSuite8) TestIndexScan(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1877,7 +1893,7 @@ func (s *testSuiteP1) TestGeneratedColumnRead(c *C) { } } -func (s *testSuiteP1) TestToPBExpr(c *C) { +func (s *testSuiteP2) TestToPBExpr(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1925,7 +1941,7 @@ func (s *testSuiteP1) TestToPBExpr(c *C) { result.Check(testkit.Rows("1", "2")) } -func (s *testSuiteP1) TestDatumXAPI(c *C) { +func (s *testSuiteP2) TestDatumXAPI(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1950,7 +1966,7 @@ func (s *testSuiteP1) TestDatumXAPI(c *C) { result.Check(testkit.Rows("11:11:12.000 11:11:12", "11:11:13.000 11:11:13")) } -func (s *testSuiteP1) TestSQLMode(c *C) { +func (s *testSuiteP2) TestSQLMode(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2000,7 +2016,7 @@ func (s *testSuiteP1) TestSQLMode(c *C) { tk.MustExec("set @@global.sql_mode = 'STRICT_TRANS_TABLES'") } -func (s *testSuiteP1) TestTableDual(c *C) { +func (s *testSuiteP2) TestTableDual(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") result := tk.MustQuery("Select 1") @@ -2017,7 +2033,7 @@ func (s *testSuiteP1) TestTableDual(c *C) { tk.MustQuery("select t1.* from t t1, t t2 where t1.a=t2.a and 1=0").Check(testkit.Rows()) } -func (s *testSuiteP1) TestTableScan(c *C) { +func (s *testSuiteP2) TestTableScan(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use information_schema") result := tk.MustQuery("select * from schemata") @@ -2036,7 +2052,7 @@ func (s *testSuiteP1) TestTableScan(c *C) { result.Check(testkit.Rows("1")) } -func (s *testSuiteP1) TestAdapterStatement(c *C) { +func (s *testSuiteP2) TestAdapterStatement(c *C) { se, err := session.CreateSession4Test(s.store) c.Check(err, IsNil) se.GetSessionVars().TxnCtx.InfoSchema = domain.GetDomain(se).InfoSchema() @@ -2054,7 +2070,7 @@ func (s *testSuiteP1) TestAdapterStatement(c *C) { c.Check(stmt.OriginText(), Equals, "create table test.t (a int)") } -func (s *testSuiteP1) TestIsPointGet(c *C) { +func (s *testSuiteP2) TestIsPointGet(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use mysql") ctx := tk.Se.(sessionctx.Context) @@ -2079,7 +2095,7 @@ func (s *testSuiteP1) TestIsPointGet(c *C) { } } -func (s *testSuiteP1) TestPointGetRepeatableRead(c *C) { +func (s *testSuiteP2) TestPointGetRepeatableRead(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") tk1.MustExec(`create table point_get (a int, b int, c int, @@ -2115,7 +2131,7 @@ func (s *testSuiteP1) TestPointGetRepeatableRead(c *C) { c.Assert(failpoint.Disable(step2), IsNil) } -func (s *testSuiteP1) TestBatchPointGetRepeatableRead(c *C) { +func (s *testSuiteP2) TestBatchPointGetRepeatableRead(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") tk1.MustExec(`create table batch_point_get (a int, b int, c int, unique key k_b(a, b, c))`) @@ -2149,7 +2165,7 @@ func (s *testSuiteP1) TestBatchPointGetRepeatableRead(c *C) { c.Assert(failpoint.Disable(step2), IsNil) } -func (s *testSuite4) TestSplitRegionTimeout(c *C) { +func (s *testSuite7) TestSplitRegionTimeout(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockSplitRegionTimeout", `return(true)`), IsNil) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -2167,7 +2183,7 @@ func (s *testSuite4) TestSplitRegionTimeout(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockScatterRegionTimeout"), IsNil) } -func (s *testSuiteP1) TestRow(c *C) { +func (s *testSuiteP2) TestRow(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2216,7 +2232,7 @@ func (s *testSuiteP1) TestRow(c *C) { result.Check(testkit.Rows("1")) } -func (s *testSuiteP1) TestColumnName(c *C) { +func (s *testSuiteP2) TestColumnName(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2304,7 +2320,7 @@ func (s *testSuiteP1) TestColumnName(c *C) { c.Assert(fields[0].ColumnAsName.L, Equals, "if(1,c,c)") } -func (s *testSuiteP1) TestSelectVar(c *C) { +func (s *testSuiteP2) TestSelectVar(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2319,7 +2335,7 @@ func (s *testSuiteP1) TestSelectVar(c *C) { tk.MustExec("select SQL_BUFFER_RESULT d from t group by d") } -func (s *testSuiteP1) TestHistoryRead(c *C) { +func (s *testSuiteP2) TestHistoryRead(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists history_read") @@ -2383,7 +2399,7 @@ func (s *testSuiteP1) TestHistoryRead(c *C) { tk.MustQuery("select * from history_read order by a").Check(testkit.Rows("2 ", "4 ", "8 8", "9 9")) } -func (s *testSuiteP1) TestLowResolutionTSORead(c *C) { +func (s *testSuite2) TestLowResolutionTSORead(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@autocommit=1") tk.MustExec("use test") @@ -2823,7 +2839,7 @@ func (s *testSuite1) TearDownTest(c *C) { } } -func (s *testSuite1) TestAddIndexPriority(c *C) { +func (s *testSuite2) TestAddIndexPriority(c *C) { cli := &checkRequestClient{} hijackClient := func(c tikv.Client) tikv.Client { cli.Client = c @@ -3814,7 +3830,7 @@ func (s *testSuite) TestSelectView(c *C) { err = tk.ExecToErr("select * from view2") c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'test.view_t.a' in 'field list'") err = tk.ExecToErr("select * from view3") - c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'a' in 'field list'") + c.Assert(err.Error(), Equals, plannercore.ErrViewInvalid.GenWithStackByArgs("test", "view3").Error()) tk.MustExec("drop table view_t;") tk.MustExec("create table view_t(a int,b int,c int)") tk.MustExec("insert into view_t values(1,2,3)") @@ -3843,43 +3859,28 @@ func (s *testSuite) TestSelectView(c *C) { } type testSuite2 struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain - *parser.Parser - ctx *mock.Context + *baseTestSuite } -func (s *testSuite2) SetUpSuite(c *C) { - s.Parser = parser.New() - flag.Lookup("mockTikv") - useMockTikv := *mockTikv - if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() - store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), - ) - c.Assert(err, IsNil) - s.store = store - session.SetSchemaLease(0) - session.DisableStats4Test() +func (s *testSuite2) TearDownTest(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show full tables") + for _, tb := range r.Rows() { + tableName := tb[0] + if tb[1] == "VIEW" { + tk.MustExec(fmt.Sprintf("drop view %v", tableName)) + } else { + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } } - d, err := session.BootstrapSession(s.store) - c.Assert(err, IsNil) - d.SetStatsUpdating(true) - s.domain = d } -func (s *testSuite2) TearDownSuite(c *C) { - s.domain.Close() - s.store.Close() +type testSuite3 struct { + *baseTestSuite } -func (s *testSuite2) TearDownTest(c *C) { +func (s *testSuite3) TearDownTest(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") r := tk.MustQuery("show full tables") @@ -3893,44 +3894,29 @@ func (s *testSuite2) TearDownTest(c *C) { } } -type testSuite3 struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain - *parser.Parser - ctx *mock.Context +type testSuite4 struct { + *baseTestSuite } -func (s *testSuite3) SetUpSuite(c *C) { - s.Parser = parser.New() - flag.Lookup("mockTikv") - useMockTikv := *mockTikv - if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() - store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), - ) - c.Assert(err, IsNil) - s.store = store - session.SetSchemaLease(0) - session.DisableStats4Test() +func (s *testSuite4) TearDownTest(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show full tables") + for _, tb := range r.Rows() { + tableName := tb[0] + if tb[1] == "VIEW" { + tk.MustExec(fmt.Sprintf("drop view %v", tableName)) + } else { + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } } - d, err := session.BootstrapSession(s.store) - c.Assert(err, IsNil) - d.SetStatsUpdating(true) - s.domain = d } -func (s *testSuite3) TearDownSuite(c *C) { - s.domain.Close() - s.store.Close() +type testSuite5 struct { + *baseTestSuite } -func (s *testSuite3) TearDownTest(c *C) { +func (s *testSuite5) TearDownTest(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") r := tk.MustQuery("show full tables") @@ -3944,44 +3930,47 @@ func (s *testSuite3) TearDownTest(c *C) { } } -type testSuite4 struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain - *parser.Parser - ctx *mock.Context +type testSuite6 struct { + *baseTestSuite } -func (s *testSuite4) SetUpSuite(c *C) { - s.Parser = parser.New() - flag.Lookup("mockTikv") - useMockTikv := *mockTikv - if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() - store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), - ) - c.Assert(err, IsNil) - s.store = store - session.SetSchemaLease(0) - session.DisableStats4Test() +func (s *testSuite6) TearDownTest(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show full tables") + for _, tb := range r.Rows() { + tableName := tb[0] + if tb[1] == "VIEW" { + tk.MustExec(fmt.Sprintf("drop view %v", tableName)) + } else { + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } } - d, err := session.BootstrapSession(s.store) - c.Assert(err, IsNil) - d.SetStatsUpdating(true) - s.domain = d } -func (s *testSuite4) TearDownSuite(c *C) { - s.domain.Close() - s.store.Close() +type testSuite7 struct { + *baseTestSuite } -func (s *testSuite4) TearDownTest(c *C) { +func (s *testSuite7) TearDownTest(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show full tables") + for _, tb := range r.Rows() { + tableName := tb[0] + if tb[1] == "VIEW" { + tk.MustExec(fmt.Sprintf("drop view %v", tableName)) + } else { + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } + } +} + +type testSuite8 struct { + *baseTestSuite +} + +func (s *testSuite8) TearDownTest(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") r := tk.MustQuery("show full tables") @@ -3995,7 +3984,7 @@ func (s *testSuite4) TearDownTest(c *C) { } } -func (s *testSuiteP1) TestStrToDateBuiltin(c *C) { +func (s *testSuiteP2) TestStrToDateBuiltin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery(`select str_to_date('20190101','%Y%m%d%!') from dual`).Check(testkit.Rows("2019-01-01")) tk.MustQuery(`select str_to_date('20190101','%Y%m%d%f') from dual`).Check(testkit.Rows("2019-01-01 00:00:00.000000")) @@ -4038,7 +4027,7 @@ func (s *testSuiteP1) TestStrToDateBuiltin(c *C) { tk.MustQuery(`select str_to_date('18_10_22','%y_%m_%d') from dual`).Check(testkit.Rows("2018-10-22")) } -func (s *testSuiteP1) TestReadPartitionedTable(c *C) { +func (s *testSuiteP2) TestReadPartitionedTable(c *C) { // Test three reader on partitioned table. tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -4055,7 +4044,7 @@ func (s *testSuiteP1) TestReadPartitionedTable(c *C) { tk.MustQuery("select a from pt where b = 3").Check(testkit.Rows("3")) } -func (s *testSuiteP1) TestSplitRegion(c *C) { +func (s *testSuiteP2) TestSplitRegion(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t, t1") @@ -4387,7 +4376,7 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta return tbl } -func (s *testSuiteP1) TestIssue10435(c *C) { +func (s *testSuiteP2) TestIssue10435(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -4401,7 +4390,7 @@ func (s *testSuiteP1) TestIssue10435(c *C) { ) } -func (s *testSuiteP1) TestUnsignedFeedback(c *C) { +func (s *testSuiteP2) TestUnsignedFeedback(c *C) { tk := testkit.NewTestKit(c, s.store) oriProbability := statistics.FeedbackProbability.Load() statistics.FeedbackProbability.Store(1.0) @@ -4476,9 +4465,10 @@ func (s *testOOMSuite) TestDistSQLMemoryControl(c *C) { } func setOOMAction(action string) { - newConf := config.NewConfig() + old := config.GetGlobalConfig() + newConf := *old newConf.OOMAction = action - config.StoreGlobalConfig(newConf) + config.StoreGlobalConfig(&newConf) } func (s *testSuite) TestOOMPanicAction(c *C) { @@ -4572,7 +4562,7 @@ func (s *testRecoverTable) SetUpSuite(c *C) { func (s *testRecoverTable) TestRecoverTable(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") }() tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_recover") @@ -4668,7 +4658,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { c.Assert(gcEnable, Equals, false) } -func (s *testSuiteP1) TestPointGetPreparedPlan(c *C) { +func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists ps_text") defer tk1.MustExec("drop database if exists ps_text") @@ -4822,7 +4812,7 @@ func (s *testSuiteP1) TestPointGetPreparedPlan(c *C) { tk1.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("3 3 3 10")) } -func (s *testSuiteP1) TestPointGetPreparedPlanWithCommitMode(c *C) { +func (s *testSuiteP2) TestPointGetPreparedPlanWithCommitMode(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists ps_text") defer tk1.MustExec("drop database if exists ps_text") @@ -4885,7 +4875,7 @@ func (s *testSuiteP1) TestPointGetPreparedPlanWithCommitMode(c *C) { tk2.MustQuery("select * from t where a = 1").Check(testkit.Rows("1 1 11")) } -func (s *testSuiteP1) TestPointUpdatePreparedPlan(c *C) { +func (s *testSuiteP2) TestPointUpdatePreparedPlan(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists pu_test") defer tk1.MustExec("drop database if exists pu_test") @@ -4986,7 +4976,7 @@ func (s *testSuiteP1) TestPointUpdatePreparedPlan(c *C) { tk1.MustQuery("select * from t where a = 2").Check(testkit.Rows("2 2 2 10")) } -func (s *testSuiteP1) TestPointUpdatePreparedPlanWithCommitMode(c *C) { +func (s *testSuiteP2) TestPointUpdatePreparedPlanWithCommitMode(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists pu_test2") defer tk1.MustExec("drop database if exists pu_test2") diff --git a/executor/explain_test.go b/executor/explain_test.go index 89aaa2810e637..626b206403a1c 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -153,7 +153,7 @@ func (s *testSuite1) checkMemoryInfo(c *C, tk *testkit.TestKit, sql string) { } } -func (s *testSuite1) TestExplainAnalyzeExecutionInfo(c *C) { +func (s *testSuite2) TestExplainAnalyzeExecutionInfo(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") tk.MustExec("create table t (v int, k int, key(k))") @@ -187,7 +187,7 @@ func (s *testSuite1) TestExplainAnalyzeExecutionInfo(c *C) { tk.MustExec("drop table if exists lineitem") } -func (s *testSuite1) checkExecutionInfo(c *C, tk *testkit.TestKit, sql string) { +func (s *testSuite2) checkExecutionInfo(c *C, tk *testkit.TestKit, sql string) { executionInfoCol := 4 rows := tk.MustQuery(sql).Rows() for _, row := range rows { diff --git a/executor/grant.go b/executor/grant.go index 8f713dbddcaf9..66157d591b9b3 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -302,8 +302,8 @@ func (e *GrantExec) grantColumnPriv(priv *ast.PrivElem, user *ast.UserSpec) erro func composeGlobalPrivUpdate(priv mysql.PrivilegeType, value string) (string, error) { if priv == mysql.AllPriv { strs := make([]string, 0, len(mysql.Priv2UserCol)) - for _, v := range mysql.Priv2UserCol { - strs = append(strs, fmt.Sprintf(`%s='%s'`, v, value)) + for _, v := range mysql.AllGlobalPrivs { + strs = append(strs, fmt.Sprintf(`%s='%s'`, mysql.Priv2UserCol[v], value)) } return strings.Join(strs, ", "), nil } diff --git a/executor/grant_test.go b/executor/grant_test.go index 0016e0b96c6f1..f652ec37a1448 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -53,6 +53,12 @@ func (s *testSuite3) TestGrantGlobal(c *C) { sql := fmt.Sprintf("SELECT %s FROM mysql.User WHERE User=\"testGlobal1\" and host=\"localhost\"", mysql.Priv2UserCol[v]) tk.MustQuery(sql).Check(testkit.Rows("Y")) } + //with grant option + tk.MustExec("GRANT ALL ON *.* TO 'testGlobal1'@'localhost' WITH GRANT OPTION;") + for _, v := range mysql.AllGlobalPrivs { + sql := fmt.Sprintf("SELECT %s FROM mysql.User WHERE User=\"testGlobal1\" and host=\"localhost\"", mysql.Priv2UserCol[v]) + tk.MustQuery(sql).Check(testkit.Rows("Y")) + } } func (s *testSuite3) TestGrantDBScope(c *C) { @@ -96,6 +102,13 @@ func (s *testSuite3) TestWithGrantOption(c *C) { // Grant select priv to the user, with grant option. tk.MustExec("GRANT select ON test.* TO 'testWithGrant'@'localhost' WITH GRANT OPTION;") tk.MustQuery("SELECT grant_priv FROM mysql.DB WHERE User=\"testWithGrant\" and host=\"localhost\" and db=\"test\"").Check(testkit.Rows("Y")) + + tk.MustExec("CREATE USER 'testWithGrant1'") + tk.MustQuery("SELECT grant_priv FROM mysql.user WHERE User=\"testWithGrant1\"").Check(testkit.Rows("N")) + tk.MustExec("GRANT ALL ON *.* TO 'testWithGrant1'") + tk.MustQuery("SELECT grant_priv FROM mysql.user WHERE User=\"testWithGrant1\"").Check(testkit.Rows("N")) + tk.MustExec("GRANT ALL ON *.* TO 'testWithGrant1' WITH GRANT OPTION") + tk.MustQuery("SELECT grant_priv FROM mysql.user WHERE User=\"testWithGrant1\"").Check(testkit.Rows("Y")) } func (s *testSuite3) TestTableScope(c *C) { @@ -124,7 +137,7 @@ func (s *testSuite3) TestTableScope(c *C) { tk.MustExec("USE test;") tk.MustExec(`CREATE TABLE test2(c1 int);`) // Grant all table scope privs. - tk.MustExec("GRANT ALL ON test2 TO 'testTbl1'@'localhost';") + tk.MustExec("GRANT ALL ON test2 TO 'testTbl1'@'localhost' WITH GRANT OPTION;") // Make sure all the table privs for granted user are in the Table_priv set. for _, v := range mysql.AllTablePrivs { rows := tk.MustQuery(`SELECT Table_priv FROM mysql.Tables_priv WHERE User="testTbl1" and host="localhost" and db="test" and Table_name="test2";`).Rows() diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 2a4e05fd8472f..f7352f35d47c9 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -107,7 +107,7 @@ func (s *testSuite) TestIndexJoinOverflow(c *C) { tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a;`).Check(testkit.Rows()) } -func (s *testSuite2) TestIssue11061(c *C) { +func (s *testSuite5) TestIssue11061(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(c varchar(30), index ix_c(c(10)))") @@ -115,7 +115,7 @@ func (s *testSuite2) TestIssue11061(c *C) { tk.MustQuery("SELECT /*+ TIDB_INLJ(t1) */ SUM(LENGTH(c)) FROM t1 WHERE c IN (SELECT t1.c FROM t1)").Check(testkit.Rows("20")) } -func (s *testSuite2) TestIndexJoinPartitionTable(c *C) { +func (s *testSuite5) TestIndexJoinPartitionTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int not null, c int, key idx(c)) partition by hash(b) partitions 30") diff --git a/executor/insert.go b/executor/insert.go index c79f4bc1071a0..fcdc6cd8ccb38 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -34,8 +35,12 @@ import ( // InsertExec represents an insert executor. type InsertExec struct { *InsertValues - OnDuplicate []*expression.Assignment - Priority mysql.PriorityEnum + OnDuplicate []*expression.Assignment + evalBuffer4Dup chunk.MutRow + curInsertVals chunk.MutRow + row4Update []types.Datum + + Priority mysql.PriorityEnum } func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error { @@ -261,6 +266,9 @@ func (e *InsertExec) Close() error { // Open implements the Executor Open interface. func (e *InsertExec) Open(ctx context.Context) error { + if e.OnDuplicate != nil { + e.initEvalBuffer4Dup() + } if e.SelectExec != nil { return e.SelectExec.Open(ctx) } @@ -270,31 +278,60 @@ func (e *InsertExec) Open(ctx context.Context) error { return nil } +func (e *InsertExec) initEvalBuffer4Dup() { + // Use public columns for new row. + numCols := len(e.Table.Cols()) + // Use writable columns for old row for update. + numWritableCols := len(e.Table.WritableCols()) + + evalBufferTypes := make([]*types.FieldType, 0, numCols+numWritableCols) + + // Append the old row before the new row, to be consistent with "Schema4OnDuplicate" in the "Insert" PhysicalPlan. + for _, col := range e.Table.WritableCols() { + evalBufferTypes = append(evalBufferTypes, &col.FieldType) + } + for _, col := range e.Table.Cols() { + evalBufferTypes = append(evalBufferTypes, &col.FieldType) + } + if e.hasExtraHandle { + evalBufferTypes = append(evalBufferTypes, types.NewFieldType(mysql.TypeLonglong)) + } + e.evalBuffer4Dup = chunk.MutRowFromTypes(evalBufferTypes) + e.curInsertVals = chunk.MutRowFromTypes(evalBufferTypes[numWritableCols:]) + e.row4Update = make([]types.Datum, 0, len(evalBufferTypes)) +} + // doDupRowUpdate updates the duplicate row. func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle int64, oldRow []types.Datum, newRow []types.Datum, cols []*expression.Assignment) ([]types.Datum, bool, int64, error) { assignFlag := make([]bool, len(e.Table.WritableCols())) // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values - e.ctx.GetSessionVars().CurrInsertValues = chunk.MutRowFromDatums(newRow).ToRow() + e.curInsertVals.SetDatums(newRow...) + e.ctx.GetSessionVars().CurrInsertValues = e.curInsertVals.ToRow() // NOTE: In order to execute the expression inside the column assignment, // we have to put the value of "oldRow" before "newRow" in "row4Update" to // be consistent with "Schema4OnDuplicate" in the "Insert" PhysicalPlan. - row4Update := make([]types.Datum, 0, len(oldRow)+len(newRow)) - row4Update = append(row4Update, oldRow...) - row4Update = append(row4Update, newRow...) + e.row4Update = e.row4Update[:0] + e.row4Update = append(e.row4Update, oldRow...) + e.row4Update = append(e.row4Update, newRow...) // Update old row when the key is duplicated. + e.evalBuffer4Dup.SetDatums(e.row4Update...) for _, col := range cols { - val, err1 := col.Expr.Eval(chunk.MutRowFromDatums(row4Update).ToRow()) + val, err1 := col.Expr.Eval(e.evalBuffer4Dup.ToRow()) + if err1 != nil { + return nil, false, 0, err1 + } + e.row4Update[col.Col.Index], err1 = table.CastValue(e.ctx, val, col.Col.ToInfo()) if err1 != nil { return nil, false, 0, err1 } - row4Update[col.Col.Index] = val + e.evalBuffer4Dup.SetDatum(col.Col.Index, e.row4Update[col.Col.Index]) assignFlag[col.Col.Index] = true } - newData := row4Update[:len(oldRow)] + newData := e.row4Update[:len(oldRow)] _, handleChanged, newHandle, err := updateRecord(ctx, e.ctx, handle, oldRow, newData, assignFlag, e.Table, true) if err != nil { return nil, false, 0, err diff --git a/executor/insert_test.go b/executor/insert_test.go index 1a8e9c7de99eb..a2b8418451062 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite3) TestInsertOnDuplicateKey(c *C) { +func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/join_test.go b/executor/join_test.go index ff39a751ad6f6..b82667a93ea53 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -27,7 +27,19 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite2) TestJoinPanic(c *C) { +type testSuiteJoin1 struct { + *baseTestSuite +} + +type testSuiteJoin2 struct { + *baseTestSuite +} + +type testSuiteJoin3 struct { + *baseTestSuite +} + +func (s *testSuiteJoin1) TestJoinPanic(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") @@ -38,11 +50,11 @@ func (s *testSuite2) TestJoinPanic(c *C) { c.Check(err, NotNil) } -func (s *testSuite2) TestJoinInDisk(c *C) { +func (s *testSuite) TestJoinInDisk(c *C) { originCfg := config.GetGlobalConfig() - newConf := config.NewConfig() + newConf := *originCfg newConf.OOMUseTmpStorage = true - config.StoreGlobalConfig(newConf) + config.StoreGlobalConfig(&newConf) defer config.StoreGlobalConfig(originCfg) tk := testkit.NewTestKit(c, s.store) @@ -66,7 +78,7 @@ func (s *testSuite2) TestJoinInDisk(c *C) { result.Check(testkit.Rows("2 2 2 3")) } -func (s *testSuite2) TestJoin(c *C) { +func (s *testSuiteJoin2) TestJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_index_lookup_join_concurrency = 200") @@ -303,7 +315,7 @@ func (s *testSuite2) TestJoin(c *C) { tk.MustQuery("select min(t2.b) from t1 right join t2 on t2.a=t1.a right join t3 on t2.a=t3.a left join t4 on t3.a=t4.a").Check(testkit.Rows("1")) } -func (s *testSuite2) TestJoinCast(c *C) { +func (s *testSuiteJoin2) TestJoinCast(c *C) { tk := testkit.NewTestKit(c, s.store) var result *testkit.Result @@ -482,7 +494,7 @@ func (s *testSuite2) TestJoinCast(c *C) { tk.MustExec("set @@tidb_init_chunk_size=32") } -func (s *testSuite2) TestUsing(c *C) { +func (s *testSuiteJoin3) TestUsing(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -530,7 +542,7 @@ func (s *testSuite2) TestUsing(c *C) { tk.MustExec("select * from t join tt using(a)") } -func (s *testSuite2) TestNaturalJoin(c *C) { +func (s *testSuiteJoin1) TestNaturalJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -545,7 +557,7 @@ func (s *testSuite2) TestNaturalJoin(c *C) { tk.MustQuery("select * from t1 natural right join t2 order by a").Check(testkit.Rows("1 3 2", "100 200 ")) } -func (s *testSuite2) TestMultiJoin(c *C) { +func (s *testSuiteJoin3) TestMultiJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t35(a35 int primary key, b35 int, x35 int)") @@ -637,7 +649,7 @@ AND b44=a42`) result.Check(testkit.Rows("7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7")) } -func (s *testSuite2) TestSubquerySameTable(c *C) { +func (s *testSuiteJoin3) TestSubquerySameTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -649,7 +661,7 @@ func (s *testSuite2) TestSubquerySameTable(c *C) { result.Check(testkit.Rows("1")) } -func (s *testSuite2) TestSubquery(c *C) { +func (s *testSuiteJoin3) TestSubquery(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("set @@tidb_hashagg_partial_concurrency=1") @@ -821,7 +833,7 @@ func (s *testSuite2) TestSubquery(c *C) { tk.MustExec("set @@tidb_hash_join_concurrency=5") } -func (s *testSuite2) TestInSubquery(c *C) { +func (s *testSuiteJoin1) TestInSubquery(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -888,7 +900,7 @@ func (s *testSuite2) TestInSubquery(c *C) { result.Check(testkit.Rows("2", "2", "1")) } -func (s *testSuite2) TestJoinLeak(c *C) { +func (s *testSuiteJoin1) TestJoinLeak(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("use test") @@ -910,7 +922,7 @@ func (s *testSuite2) TestJoinLeak(c *C) { tk.MustExec("set @@tidb_hash_join_concurrency=5") } -func (s *testSuite2) TestHashJoinExecEncodeDecodeRow(c *C) { +func (s *testSuiteJoin1) TestHashJoinExecEncodeDecodeRow(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -923,7 +935,7 @@ func (s *testSuite2) TestHashJoinExecEncodeDecodeRow(c *C) { result.Check(testkit.Rows("2003-06-09 10:51:26")) } -func (s *testSuite2) TestSubqueryInJoinOn(c *C) { +func (s *testSuiteJoin1) TestSubqueryInJoinOn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -937,7 +949,7 @@ func (s *testSuite2) TestSubqueryInJoinOn(c *C) { c.Check(err, NotNil) } -func (s *testSuite2) TestIssue5255(c *C) { +func (s *testSuiteJoin1) TestIssue5255(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -948,7 +960,7 @@ func (s *testSuite2) TestIssue5255(c *C) { tk.MustQuery("select /*+ TIDB_INLJ(t1) */ * from t1 join t2 on t1.a=t2.a").Check(testkit.Rows("1 2017-11-29 2.2 1")) } -func (s *testSuite2) TestIssue5278(c *C) { +func (s *testSuiteJoin1) TestIssue5278(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t, tt") @@ -958,7 +970,7 @@ func (s *testSuite2) TestIssue5278(c *C) { tk.MustQuery("select * from t left join tt on t.a=tt.a left join t ttt on t.a=ttt.a").Check(testkit.Rows("1 1 1 1")) } -func (s *testSuite2) TestIndexLookupJoin(c *C) { +func (s *testSuiteJoin1) TestIndexLookupJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set @@tidb_init_chunk_size=2") @@ -1020,7 +1032,7 @@ func (s *testSuite2) TestIndexLookupJoin(c *C) { )) } -func (s *testSuite2) TestIndexNestedLoopHashJoin(c *C) { +func (s *testSuiteJoin1) TestIndexNestedLoopHashJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set @@tidb_init_chunk_size=2") @@ -1054,7 +1066,7 @@ func (s *testSuite2) TestIndexNestedLoopHashJoin(c *C) { } } -func (s *testSuite2) TestMergejoinOrder(c *C) { +func (s *testSuiteJoin3) TestMergejoinOrder(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") @@ -1099,7 +1111,7 @@ func (s *testSuite2) TestMergejoinOrder(c *C) { )) } -func (s *testSuite2) TestEmbeddedOuterJoin(c *C) { +func (s *testSuiteJoin1) TestEmbeddedOuterJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -1110,7 +1122,7 @@ func (s *testSuite2) TestEmbeddedOuterJoin(c *C) { Check(testkit.Rows("1 1 ")) } -func (s *testSuite2) TestHashJoin(c *C) { +func (s *testSuiteJoin1) TestHashJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -1137,7 +1149,7 @@ func (s *testSuite2) TestHashJoin(c *C) { c.Assert(innerExecInfo[strings.Index(innerExecInfo, "rows")+5:strings.Index(innerExecInfo, "rows")+6], Equals, "0") } -func (s *testSuite2) TestJoinDifferentDecimals(c *C) { +func (s *testSuiteJoin1) TestJoinDifferentDecimals(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("Use test") tk.MustExec("Drop table if exists t1") @@ -1156,7 +1168,7 @@ func (s *testSuite2) TestJoinDifferentDecimals(c *C) { rst.Check(testkit.Rows("1 1.000", "2 2.000", "3 3.000")) } -func (s *testSuite2) TestNullEmptyAwareSemiJoin(c *C) { +func (s *testSuiteJoin3) TestNullEmptyAwareSemiJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1529,7 +1541,7 @@ func (s *testSuite2) TestNullEmptyAwareSemiJoin(c *C) { } } -func (s *testSuite2) TestScalarFuncNullSemiJoin(c *C) { +func (s *testSuiteJoin1) TestScalarFuncNullSemiJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1545,7 +1557,7 @@ func (s *testSuite2) TestScalarFuncNullSemiJoin(c *C) { tk.MustQuery("select a in (select a+b from s) from t").Check(testkit.Rows("", "")) } -func (s *testSuite2) TestInjectProjOnTopN(c *C) { +func (s *testSuiteJoin1) TestInjectProjOnTopN(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -1558,7 +1570,7 @@ func (s *testSuite2) TestInjectProjOnTopN(c *C) { )) } -func (s *testSuite2) TestIssue11544(c *C) { +func (s *testSuiteJoin1) TestIssue11544(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table 11544t(a int)") @@ -1569,7 +1581,7 @@ func (s *testSuite2) TestIssue11544(c *C) { tk.MustQuery("select /*+ TIDB_INLJ(tt) */ * from 11544t t, 11544tt tt where t.a=tt.a and tt.b in ('aaaaaaa', 'aaaabbb', 'aaaacccc')").Check(testkit.Rows("1 1 aaaaaaa", "1 1 aaaabbb", "1 1 aaaacccc")) } -func (s *testSuite2) TestIssue11390(c *C) { +func (s *testSuiteJoin1) TestIssue11390(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table 11390t (k1 int unsigned, k2 int unsigned, key(k1, k2))") diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 8267466bbb980..0a32a8eb92d73 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -235,7 +235,7 @@ func checkPlanAndRun(tk *testkit.TestKit, c *C, plan string, sql string) *testki return tk.MustQuery(sql) } -func (s *testSuite1) TestMergeJoin(c *C) { +func (s *testSuite2) TestMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -362,7 +362,7 @@ func (s *testSuite1) TestMergeJoin(c *C) { )) } -func (s *testSuite1) Test3WaysMergeJoin(c *C) { +func (s *testSuite2) Test3WaysMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -387,7 +387,7 @@ func (s *testSuite1) Test3WaysMergeJoin(c *C) { result.Check(testkit.Rows("2 2 2 3 2 4", "3 3 3 4 3 10")) } -func (s *testSuite1) TestMergeJoinDifferentTypes(c *C) { +func (s *testSuite2) TestMergeJoinDifferentTypes(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t1;`) diff --git a/executor/metrics_test.go b/executor/metrics_test.go index 662858fd9928c..cea19a2b12ee3 100644 --- a/executor/metrics_test.go +++ b/executor/metrics_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite4) TestStmtLabel(c *C) { +func (s *testSuite7) TestStmtLabel(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table label (c1 int primary key, c2 int, c3 int, index (c2))") diff --git a/executor/projection.go b/executor/projection.go index e6423b03b2af8..70de60f3dab66 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -77,7 +77,10 @@ func (e *ProjectionExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } + return e.open(ctx) +} +func (e *ProjectionExec) open(ctx context.Context) error { e.prepared = false e.parentReqRows = int64(e.maxChunkSize) diff --git a/executor/set_test.go b/executor/set_test.go index 723d91dfa643b..a4b96c4f07800 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/util/testutil" ) -func (s *testSuite2) TestSetVar(c *C) { +func (s *testSuite5) TestSetVar(c *C) { tk := testkit.NewTestKit(c, s.store) testSQL := "SET @a = 1;" tk.MustExec(testSQL) @@ -267,8 +267,8 @@ func (s *testSuite2) TestSetVar(c *C) { tk.MustExec("set tidb_slow_log_threshold = 0") tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("0")) - tk.MustExec("set tidb_slow_log_threshold = 1") - tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("1")) + tk.MustExec("set tidb_slow_log_threshold = 30000") + tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("30000")) _, err = tk.Exec("set global tidb_slow_log_threshold = 0") c.Assert(err, NotNil) @@ -390,7 +390,7 @@ func (s *testSuite2) TestSetVar(c *C) { tk.MustQuery("select @@tidb_record_plan_in_slow_log;").Check(testkit.Rows("0")) } -func (s *testSuite2) TestSetCharset(c *C) { +func (s *testSuite5) TestSetCharset(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`SET NAMES latin1`) @@ -415,7 +415,7 @@ func (s *testSuite2) TestSetCharset(c *C) { tk.MustExec(`SET NAMES binary`) } -func (s *testSuite2) TestValidateSetVar(c *C) { +func (s *testSuite5) TestValidateSetVar(c *C) { tk := testkit.NewTestKit(c, s.store) _, err := tk.Exec("set global tidb_distsql_scan_concurrency='fff';") @@ -768,7 +768,7 @@ func (s *testSuite2) TestValidateSetVar(c *C) { c.Assert(terror.ErrorEqual(err, variable.ErrUnsupportedValueForVar), IsTrue, Commentf("err %v", err)) } -func (s *testSuite2) TestSelectGlobalVar(c *C) { +func (s *testSuite5) TestSelectGlobalVar(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("151")) @@ -788,7 +788,7 @@ func (s *testSuite2) TestSelectGlobalVar(c *C) { c.Assert(terror.ErrorEqual(err, variable.UnknownSystemVar), IsTrue, Commentf("err %v", err)) } -func (s *testSuite2) TestEnableNoopFunctionsVar(c *C) { +func (s *testSuite5) TestEnableNoopFunctionsVar(c *C) { tk := testkit.NewTestKit(c, s.store) // test for tidb_enable_noop_functions diff --git a/executor/show_test.go b/executor/show_test.go index e8a5256763453..6c099b83c18d0 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/util/testutil" ) -func (s *testSuite2) TestShowVisibility(c *C) { +func (s *testSuite5) TestShowVisibility(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database showdatabase") tk.MustExec("use showdatabase") @@ -73,7 +73,7 @@ func (s *testSuite2) TestShowVisibility(c *C) { tk.MustExec("drop database showdatabase") } -func (s *testSuite2) TestShowDatabasesInfoSchemaFirst(c *C) { +func (s *testSuite5) TestShowDatabasesInfoSchemaFirst(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery("show databases").Check(testkit.Rows("INFORMATION_SCHEMA")) tk.MustExec(`create user 'show'@'%'`) @@ -97,7 +97,7 @@ func (s *testSuite2) TestShowDatabasesInfoSchemaFirst(c *C) { tk.MustExec(`drop database BBBB`) } -func (s *testSuite2) TestShowWarnings(c *C) { +func (s *testSuite5) TestShowWarnings(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testSQL := `create table if not exists show_warnings (a int)` @@ -128,7 +128,7 @@ func (s *testSuite2) TestShowWarnings(c *C) { tk.MustQuery("select @@warning_count").Check(testutil.RowsWithSep("|", "0")) } -func (s *testSuite2) TestShowErrors(c *C) { +func (s *testSuite5) TestShowErrors(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testSQL := `create table if not exists show_errors (a int)` @@ -139,7 +139,7 @@ func (s *testSuite2) TestShowErrors(c *C) { tk.MustQuery("show errors").Check(testutil.RowsWithSep("|", "Error|1050|Table 'test.show_errors' already exists")) } -func (s *testSuite2) TestIssue3641(c *C) { +func (s *testSuite5) TestIssue3641(c *C) { tk := testkit.NewTestKit(c, s.store) _, err := tk.Exec("show tables;") c.Assert(err.Error(), Equals, plannercore.ErrNoDB.Error()) @@ -147,7 +147,7 @@ func (s *testSuite2) TestIssue3641(c *C) { c.Assert(err.Error(), Equals, plannercore.ErrNoDB.Error()) } -func (s *testSuite2) TestIssue10549(c *C) { +func (s *testSuite5) TestIssue10549(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE DATABASE newdb;") tk.MustExec("CREATE ROLE 'app_developer';") @@ -162,7 +162,7 @@ func (s *testSuite2) TestIssue10549(c *C) { tk.MustQuery("SHOW GRANTS FOR CURRENT_USER").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'")) } -func (s *testSuite3) TestIssue11165(c *C) { +func (s *testSuite5) TestIssue11165(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE ROLE 'r_manager';") tk.MustExec("CREATE USER 'manager'@'localhost';") @@ -175,7 +175,7 @@ func (s *testSuite3) TestIssue11165(c *C) { } // TestShow2 is moved from session_test -func (s *testSuite2) TestShow2(c *C) { +func (s *testSuite5) TestShow2(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -285,15 +285,14 @@ func (s *testSuite2) TestShow2(c *C) { tk.MustQuery("show databases like 'test'").Check(testkit.Rows("test")) - AllPrivs := "Select,Insert,Update,Delete,Create,Drop,Process,References,Alter,Show Databases,Super,Execute,Index,Create User,Trigger,Create View,Show View,Create Role,Drop Role,CREATE TEMPORARY TABLES,LOCK TABLES,CREATE ROUTINE,ALTER ROUTINE,EVENT,SHUTDOWN" tk.MustExec(`grant all on *.* to 'root'@'%'`) - tk.MustQuery("show grants").Check(testkit.Rows(`GRANT ` + AllPrivs + ` ON *.* TO 'root'@'%'`)) + tk.MustQuery("show grants").Check(testkit.Rows(`GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION`)) - tk.MustQuery("show grants for current_user()").Check(testkit.Rows(`GRANT ` + AllPrivs + ` ON *.* TO 'root'@'%'`)) - tk.MustQuery("show grants for current_user").Check(testkit.Rows(`GRANT ` + AllPrivs + ` ON *.* TO 'root'@'%'`)) + tk.MustQuery("show grants for current_user()").Check(testkit.Rows(`GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION`)) + tk.MustQuery("show grants for current_user").Check(testkit.Rows(`GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION`)) } -func (s *testSuite2) TestShowCreateUser(c *C) { +func (s *testSuite5) TestShowCreateUser(c *C) { tk := testkit.NewTestKit(c, s.store) // Create a new user. tk.MustExec(`CREATE USER 'test_show_create_user'@'%' IDENTIFIED BY 'root';`) @@ -334,7 +333,7 @@ func (s *testSuite2) TestShowCreateUser(c *C) { rows.Check(testkit.Rows("CREATE USER 'check_priv'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) } -func (s *testSuite2) TestUnprivilegedShow(c *C) { +func (s *testSuite5) TestUnprivilegedShow(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE DATABASE testshow") @@ -365,7 +364,7 @@ func (s *testSuite2) TestUnprivilegedShow(c *C) { } -func (s *testSuite2) TestCollation(c *C) { +func (s *testSuite5) TestCollation(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -380,7 +379,7 @@ func (s *testSuite2) TestCollation(c *C) { c.Assert(fields[5].Column.Tp, Equals, mysql.TypeLonglong) } -func (s *testSuite2) TestShowTableStatus(c *C) { +func (s *testSuite5) TestShowTableStatus(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -420,7 +419,7 @@ func (s *testSuite2) TestShowTableStatus(c *C) { c.Assert(rows[0].GetString(16), Equals, "partitioned") } -func (s *testSuite2) TestShowSlow(c *C) { +func (s *testSuite5) TestShowSlow(c *C) { tk := testkit.NewTestKit(c, s.store) // The test result is volatile, because // 1. Slow queries is stored in domain, which may be affected by other tests. @@ -433,13 +432,13 @@ func (s *testSuite2) TestShowSlow(c *C) { tk.MustQuery(`admin show slow top all 3`) } -func (s *testSuite2) TestShowOpenTables(c *C) { +func (s *testSuite5) TestShowOpenTables(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery("show open tables") tk.MustQuery("show open tables in test") } -func (s *testSuite2) TestShowCreateTable(c *C) { +func (s *testSuite5) TestShowCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -612,7 +611,7 @@ func (s *testSuite2) TestShowCreateTable(c *C) { )) } -func (s *testSuite2) TestShowEscape(c *C) { +func (s *testSuite5) TestShowEscape(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/simple_test.go b/executor/simple_test.go index 7d48864212b86..e704d250e008b 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -429,9 +429,9 @@ func (s *testFlushSuite) TestFlushPrivilegesPanic(c *C) { defer store.Close() saveConf := config.GetGlobalConfig() - conf := config.NewConfig() + conf := *saveConf conf.Security.SkipGrantTable = true - config.StoreGlobalConfig(conf) + config.StoreGlobalConfig(&conf) dom, err := session.BootstrapSession(store) c.Assert(err, IsNil) diff --git a/executor/union_scan.go b/executor/union_scan.go index 72f7dcf33c386..fe548f78d359d 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -15,6 +15,7 @@ package executor import ( "context" + "sync" "github.com/pingcap/errors" "github.com/pingcap/parser/model" @@ -28,12 +29,17 @@ import ( // DirtyDB stores uncommitted write operations for a transaction. // It is stored and retrieved by context.Value and context.SetValue method. type DirtyDB struct { + sync.Mutex + // tables is a map whose key is tableID. tables map[int64]*DirtyTable } // GetDirtyTable gets the DirtyTable by id from the DirtyDB. func (udb *DirtyDB) GetDirtyTable(tid int64) *DirtyTable { + // The index join access the tables map parallelly. + // But the map throws panic in this case. So it's locked. + udb.Lock() dt, ok := udb.tables[tid] if !ok { dt = &DirtyTable{ @@ -43,6 +49,7 @@ func (udb *DirtyDB) GetDirtyTable(tid int64) *DirtyTable { } udb.tables[tid] = dt } + udb.Unlock() return dt } diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index e824066475bd8..88893c391e141 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite4) TestDirtyTransaction(c *C) { +func (s *testSuite7) TestDirtyTransaction(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -94,7 +94,7 @@ func (s *testSuite4) TestDirtyTransaction(c *C) { tk.MustExec("commit") } -func (s *testSuite4) TestUnionScanWithCastCondition(c *C) { +func (s *testSuite7) TestUnionScanWithCastCondition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table ta (a varchar(20))") @@ -107,7 +107,7 @@ func (s *testSuite4) TestUnionScanWithCastCondition(c *C) { tk.MustExec("rollback") } -func (s *testSuite4) TestUnionScanForMemBufferReader(c *C) { +func (s *testSuite7) TestUnionScanForMemBufferReader(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/window_test.go b/executor/window_test.go index 6f9a460d9e36d..fa89ac8bd5650 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite4) TestWindowFunctions(c *C) { +func (s *testSuite7) TestWindowFunctions(c *C) { tk := testkit.NewTestKit(c, s.store) var result *testkit.Result tk.MustExec("use test") @@ -179,7 +179,7 @@ func (s *testSuite4) TestWindowFunctions(c *C) { result.Check(testkit.Rows("1 1", "1 2", "2 1", "2 2")) } -func (s *testSuite4) TestWindowFunctionsDataReference(c *C) { +func (s *testSuite7) TestWindowFunctionsDataReference(c *C) { // see https://github.com/pingcap/tidb/issues/11614 tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/write_test.go b/executor/write_test.go index 1a4777a68c8fa..cbdd7287a78ce 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -302,9 +302,10 @@ func (s *testSuite4) TestInsert(c *C) { tk.MustExec("drop view v") } -func (s *testSuite) TestMultiBatch(c *C) { +func (s *testSuiteP2) TestMultiBatch(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.MustExec("drop table if exists t,t0") tk.MustExec("create table t0 (i int)") tk.MustExec("insert into t0 values (1), (1)") tk.MustExec("create table t (i int unique key)") @@ -1196,7 +1197,7 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { r.Check(testkit.Rows("111 2")) } -func (s *testSuite) TestUpdate(c *C) { +func (s *testSuite8) TestUpdate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") s.fillData(tk, "update_test") @@ -2187,7 +2188,7 @@ func (s *testBypassSuite) TestLatch(c *C) { } // TestIssue4067 Test issue https://github.com/pingcap/tidb/issues/4067 -func (s *testSuite4) TestIssue4067(c *C) { +func (s *testSuite7) TestIssue4067(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -2201,7 +2202,7 @@ func (s *testSuite4) TestIssue4067(c *C) { tk.MustQuery("select * from t1").Check(nil) } -func (s *testSuite4) TestInsertCalculatedValue(c *C) { +func (s *testSuite7) TestInsertCalculatedValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -2317,7 +2318,7 @@ func (s *testSuite4) TestInsertCalculatedValue(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("4 0 2")) } -func (s *testSuite4) TestDataTooLongErrMsg(c *C) { +func (s *testSuite7) TestDataTooLongErrMsg(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t(a varchar(2));") @@ -2330,7 +2331,7 @@ func (s *testSuite4) TestDataTooLongErrMsg(c *C) { c.Assert(err.Error(), Equals, "[types:1406]Data too long for column 'a' at row 1") } -func (s *testSuite4) TestUpdateSelect(c *C) { +func (s *testSuite7) TestUpdateSelect(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table msg (id varchar(8), b int, status int, primary key (id, b))") @@ -2342,7 +2343,7 @@ func (s *testSuite4) TestUpdateSelect(c *C) { tk.MustExec("admin check table msg") } -func (s *testSuite4) TestUpdateDelete(c *C) { +func (s *testSuite7) TestUpdateDelete(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host));") @@ -2362,7 +2363,7 @@ func (s *testSuite4) TestUpdateDelete(c *C) { tk.MustExec("drop table ttt") } -func (s *testSuite4) TestUpdateAffectRowCnt(c *C) { +func (s *testSuite7) TestUpdateAffectRowCnt(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table a(id int auto_increment, a int default null, primary key(id))") @@ -2381,7 +2382,7 @@ func (s *testSuite4) TestUpdateAffectRowCnt(c *C) { tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") } -func (s *testSuite4) TestReplaceLog(c *C) { +func (s *testSuite7) TestReplaceLog(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table testLog (a int not null primary key, b int unique key);`) @@ -2416,7 +2417,7 @@ func (s *testSuite4) TestReplaceLog(c *C) { // TestRebaseIfNeeded is for issue 7422. // There is no need to do the rebase when updating a record if the auto-increment ID not changed. // This could make the auto ID increasing speed slower. -func (s *testSuite4) TestRebaseIfNeeded(c *C) { +func (s *testSuite7) TestRebaseIfNeeded(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table t (a int not null primary key auto_increment, b int unique key);`) @@ -2448,7 +2449,7 @@ func (s *testSuite4) TestRebaseIfNeeded(c *C) { tk.MustQuery(`select a from t where b = 6;`).Check(testkit.Rows("30003")) } -func (s *testSuite4) TestDeferConstraintCheckForInsert(c *C) { +func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) @@ -2494,7 +2495,7 @@ func (s *testSuite4) TestDeferConstraintCheckForInsert(c *C) { c.Assert(err, NotNil) } -func (s *testSuite4) TestDefEnumInsert(c *C) { +func (s *testSuite7) TestDefEnumInsert(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table test (id int, prescription_type enum('a','b','c','d','e','f') NOT NULL, primary key(id));") @@ -2502,7 +2503,7 @@ func (s *testSuite4) TestDefEnumInsert(c *C) { tk.MustQuery("select prescription_type from test").Check(testkit.Rows("a")) } -func (s *testSuite4) TestIssue11059(c *C) { +func (s *testSuite7) TestIssue11059(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t (pk int primary key, uk int unique, v int)") tk.MustExec("insert into t values (2, 11, 215)") @@ -2511,7 +2512,7 @@ func (s *testSuite4) TestIssue11059(c *C) { c.Assert(err, NotNil) } -func (s *testSuite4) TestSetWithRefGenCol(c *C) { +func (s *testSuite7) TestSetWithRefGenCol(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table t (i int, j int as (i+1) not null);`) @@ -2554,7 +2555,7 @@ func (s *testSuite4) TestSetWithRefGenCol(c *C) { c.Assert(err, NotNil) } -func (s *testSuite4) TestSetWithCurrentTimestampAndNow(c *C) { +func (s *testSuite7) TestSetWithCurrentTimestampAndNow(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test") tk.MustExec(`drop table if exists tbl;`) diff --git a/expression/aggregation/util_test.go b/expression/aggregation/util_test.go index 89b768e8676b3..00a2f7769c335 100644 --- a/expression/aggregation/util_test.go +++ b/expression/aggregation/util_test.go @@ -6,7 +6,6 @@ import ( "github.com/pingcap/check" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" ) var _ = check.Suite(&testUtilSuite{}) @@ -15,7 +14,6 @@ type testUtilSuite struct { } func (s *testUtilSuite) TestDistinct(c *check.C) { - defer testleak.AfterTest(c)() sc := &stmtctx.StatementContext{TimeZone: time.Local} dc := createDistinctChecker(sc) tests := []struct { diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index c8246b423d8d3..1a9e79f1c8785 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -670,6 +670,7 @@ func (c *arithmeticIntDivideFunctionClass) GetFunction(ctx sessionctx.Context, a bf.Tp.Flag |= mysql.UnsignedFlag } sig := &builtinArithmeticIntDivideIntSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_IntDivideInt) return sig, nil } bf := NewBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDecimal, types.ETDecimal) @@ -677,6 +678,7 @@ func (c *arithmeticIntDivideFunctionClass) GetFunction(ctx sessionctx.Context, a bf.Tp.Flag |= mysql.UnsignedFlag } sig := &builtinArithmeticIntDivideDecimalSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_IntDivideDecimal) return sig, nil } @@ -834,6 +836,7 @@ func (c *arithmeticModFunctionClass) GetFunction(ctx sessionctx.Context, args [] bf.Tp.Flag |= mysql.UnsignedFlag } sig := &builtinArithmeticModRealSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_ModReal) return sig, nil } else if lhsEvalTp == types.ETDecimal || rhsEvalTp == types.ETDecimal { bf := NewBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) @@ -842,6 +845,7 @@ func (c *arithmeticModFunctionClass) GetFunction(ctx sessionctx.Context, args [] bf.Tp.Flag |= mysql.UnsignedFlag } sig := &builtinArithmeticModDecimalSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_ModDecimal) return sig, nil } else { bf := NewBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) @@ -849,6 +853,7 @@ func (c *arithmeticModFunctionClass) GetFunction(ctx sessionctx.Context, args [] bf.Tp.Flag |= mysql.UnsignedFlag } sig := &builtinArithmeticModIntSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_ModInt) return sig, nil } } diff --git a/expression/builtin_arithmetic_test.go b/expression/builtin_arithmetic_test.go index e7dafcbcc45d0..ec75a226a838a 100644 --- a/expression/builtin_arithmetic_test.go +++ b/expression/builtin_arithmetic_test.go @@ -21,13 +21,11 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tipb/go-tipb" ) func (s *testEvaluatorSuite) TestSetFlenDecimal4RealOrDecimal(c *C) { - defer testleak.AfterTest(c)() - ret := &types.FieldType{} a := &types.FieldType{ Decimal: 1, @@ -92,8 +90,6 @@ func (s *testEvaluatorSuite) TestSetFlenDecimal4RealOrDecimal(c *C) { } func (s *testEvaluatorSuite) TestSetFlenDecimal4Int(c *C) { - defer testleak.AfterTest(c)() - ret := &types.FieldType{} a := &types.FieldType{ Decimal: 1, @@ -119,8 +115,6 @@ func (s *testEvaluatorSuite) TestSetFlenDecimal4Int(c *C) { } func (s *testEvaluatorSuite) TestArithmeticPlus(c *C) { - defer testleak.AfterTest(c)() - // case: 1 args := []interface{}{int64(12), int64(1)} @@ -199,8 +193,6 @@ func (s *testEvaluatorSuite) TestArithmeticPlus(c *C) { } func (s *testEvaluatorSuite) TestArithmeticMinus(c *C) { - defer testleak.AfterTest(c)() - // case: 1 args := []interface{}{int64(12), int64(1)} @@ -278,7 +270,6 @@ func (s *testEvaluatorSuite) TestArithmeticMinus(c *C) { } func (s *testEvaluatorSuite) TestArithmeticMultiply(c *C) { - defer testleak.AfterTest(c)() testCases := []struct { args []interface{} expect interface{} @@ -321,7 +312,6 @@ func (s *testEvaluatorSuite) TestArithmeticMultiply(c *C) { } func (s *testEvaluatorSuite) TestArithmeticDivide(c *C) { - defer testleak.AfterTest(c)() testCases := []struct { args []interface{} expect interface{} @@ -376,6 +366,12 @@ func (s *testEvaluatorSuite) TestArithmeticDivide(c *C) { sig, err := funcs[ast.Div].GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(tc.args...))) c.Assert(err, IsNil) c.Assert(sig, NotNil) + switch sig.(type) { + case *builtinArithmeticIntDivideIntSig: + c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_IntDivideInt) + case *builtinArithmeticIntDivideDecimalSig: + c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_IntDivideDecimal) + } val, err := evalBuiltinFunc(sig, chunk.Row{}) c.Assert(err, IsNil) c.Assert(val, testutil.DatumEquals, types.NewDatum(tc.expect)) @@ -383,7 +379,6 @@ func (s *testEvaluatorSuite) TestArithmeticDivide(c *C) { } func (s *testEvaluatorSuite) TestArithmeticIntDivide(c *C) { - defer testleak.AfterTest(c)() testCases := []struct { args []interface{} expect []interface{} @@ -497,7 +492,6 @@ func (s *testEvaluatorSuite) TestArithmeticIntDivide(c *C) { } func (s *testEvaluatorSuite) TestArithmeticMod(c *C) { - defer testleak.AfterTest(c)() testCases := []struct { args []interface{} expect interface{} @@ -601,6 +595,14 @@ func (s *testEvaluatorSuite) TestArithmeticMod(c *C) { c.Assert(err, IsNil) c.Assert(sig, NotNil) val, err := evalBuiltinFunc(sig, chunk.Row{}) + switch sig.(type) { + case *builtinArithmeticModRealSig: + c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_ModReal) + case *builtinArithmeticModIntSig: + c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_ModInt) + case *builtinArithmeticModDecimalSig: + c.Assert(sig.PbCode(), Equals, tipb.ScalarFuncSig_ModDecimal) + } c.Assert(err, IsNil) c.Assert(val, testutil.DatumEquals, types.NewDatum(tc.expect)) } diff --git a/expression/builtin_arithmetic_vec.go b/expression/builtin_arithmetic_vec.go index ef6f401600bd2..29fda5ca237a7 100644 --- a/expression/builtin_arithmetic_vec.go +++ b/expression/builtin_arithmetic_vec.go @@ -583,9 +583,38 @@ func (b *builtinArithmeticPlusDecimalSig) vecEvalDecimal(input *chunk.Chunk, res } func (b *builtinArithmeticMultiplyIntUnsignedSig) vectorized() bool { - return false + return true } func (b *builtinArithmeticMultiplyIntUnsignedSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + if err := b.args[0].VecEvalInt(b.ctx, input, result); err != nil { + return err + } + n := input.NumRows() + buf, err := b.bufAllocator.get(types.ETInt, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + + if err := b.args[1].VecEvalInt(b.ctx, input, buf); err != nil { + return err + } + + x := result.Uint64s() + y := buf.Uint64s() + result.MergeNulls(buf) + var res uint64 + for i := 0; i < n; i++ { + if result.IsNull(i) { + continue + } + + res = x[i] * y[i] + if x[i] != 0 && res/x[i] != y[i] { + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", b.args[0].String(), b.args[1].String())) + } + x[i] = res + } + return nil } diff --git a/expression/builtin_arithmetic_vec_test.go b/expression/builtin_arithmetic_vec_test.go index d330216921159..fea4c5f8d58b1 100644 --- a/expression/builtin_arithmetic_vec_test.go +++ b/expression/builtin_arithmetic_vec_test.go @@ -51,6 +51,12 @@ var vecBuiltinArithmeticCases = map[string][]vecExprBenchCase{ {retEvalType: types.ETReal, childrenTypes: []types.EvalType{types.ETReal, types.ETReal}}, {retEvalType: types.ETDecimal, childrenTypes: []types.EvalType{types.ETDecimal, types.ETDecimal}}, {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, geners: []dataGenerator{&rangeInt64Gener{-10000, 10000}, &rangeInt64Gener{-10000, 10000}}}, + {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETInt, types.ETInt}, childrenFieldTypes: []*types.FieldType{{Tp: mysql.TypeInt24, Flag: mysql.UnsignedFlag}, {Tp: mysql.TypeLonglong, Flag: mysql.UnsignedFlag}}, + geners: []dataGenerator{ + &rangeInt64Gener{begin: 0, end: 10000}, + &rangeInt64Gener{begin: 0, end: 10000}, + }, + }, }, ast.Round: {}, ast.And: {}, diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go index a334275795280..8b95e678b01aa 100644 --- a/expression/builtin_cast_vec.go +++ b/expression/builtin_cast_vec.go @@ -310,11 +310,45 @@ func (b *builtinCastDurationAsIntSig) vecEvalInt(input *chunk.Chunk, result *chu } func (b *builtinCastIntAsTimeSig) vectorized() bool { - return false + return true } func (b *builtinCastIntAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + buf, err := b.bufAllocator.get(types.ETInt, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalInt(b.ctx, input, buf); err != nil { + return err + } + + result.ResizeTime(n, false) + result.MergeNulls(buf) + times := result.Times() + i64s := buf.Int64s() + stmt := b.ctx.GetSessionVars().StmtCtx + fsp := int8(b.tp.Decimal) + for i := 0; i < n; i++ { + if buf.IsNull(i) { + continue + } + tm, err := types.ParseTimeFromNum(stmt, i64s[i], b.tp.Tp, fsp) + if err != nil { + if err = handleInvalidTimeError(b.ctx, err); err != nil { + return err + } + result.SetNull(i, true) + continue + } + times[i] = tm + if b.tp.Tp == mysql.TypeDate { + // Truncate hh:mm:ss part if the type is Date. + times[i].Time = types.FromDate(tm.Time.Year(), tm.Time.Month(), tm.Time.Day(), 0, 0, 0, 0) + } + } + return nil } func (b *builtinCastRealAsJSONSig) vectorized() bool { diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index c57abe7cce48f..b5361bfeeb7ed 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -55,6 +55,7 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{ {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson}}, {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&jsonStringGener{}}}, {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETDecimal}}, + {retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETInt}}, }, } diff --git a/expression/builtin_compare_test.go b/expression/builtin_compare_test.go index 0e483c004412b..b8211870f3765 100644 --- a/expression/builtin_compare_test.go +++ b/expression/builtin_compare_test.go @@ -24,11 +24,9 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestCompareFunctionWithRefine(c *C) { - defer testleak.AfterTest(c)() tblInfo := newTestTableBuilder("").add("a", mysql.TypeLong).build() tests := []struct { exprStr string @@ -82,8 +80,6 @@ func (s *testEvaluatorSuite) TestCompareFunctionWithRefine(c *C) { } func (s *testEvaluatorSuite) TestCompare(c *C) { - defer testleak.AfterTest(c)() - intVal, uintVal, realVal, stringVal, decimalVal := 1, uint64(1), 1.1, "123", types.NewDecFromFloatForTest(123.123) timeVal := types.Time{Time: types.FromGoTime(time.Now()), Fsp: 6, Type: mysql.TypeDatetime} durationVal := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second)} @@ -167,8 +163,6 @@ func (s *testEvaluatorSuite) TestCompare(c *C) { } func (s *testEvaluatorSuite) TestCoalesce(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} expected interface{} @@ -212,7 +206,6 @@ func (s *testEvaluatorSuite) TestCoalesce(c *C) { } func (s *testEvaluatorSuite) TestIntervalFunc(c *C) { - defer testleak.AfterTest(c)() sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate sc.IgnoreTruncate = true @@ -255,8 +248,6 @@ func (s *testEvaluatorSuite) TestIntervalFunc(c *C) { } func (s *testEvaluatorSuite) TestGreatestLeastFuncs(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx originIgnoreTruncate := sc.IgnoreTruncate sc.IgnoreTruncate = true diff --git a/expression/builtin_control_test.go b/expression/builtin_control_test.go index 6635a4be63a2b..5438dfb2186e0 100644 --- a/expression/builtin_control_test.go +++ b/expression/builtin_control_test.go @@ -21,12 +21,10 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) func (s *testEvaluatorSuite) TestCaseWhen(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Arg []interface{} Ret interface{} @@ -56,7 +54,6 @@ func (s *testEvaluatorSuite) TestCaseWhen(c *C) { } func (s *testEvaluatorSuite) TestIf(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -98,7 +95,6 @@ func (s *testEvaluatorSuite) TestIf(c *C) { } func (s *testEvaluatorSuite) TestIfNull(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { arg1 interface{} arg2 interface{} diff --git a/expression/builtin_encryption_test.go b/expression/builtin_encryption_test.go index 460d7cbd6e783..8f10940e1813a 100644 --- a/expression/builtin_encryption_test.go +++ b/expression/builtin_encryption_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" - "github.com/pingcap/tidb/util/testleak" ) var cryptTests = []struct { @@ -47,7 +46,6 @@ var cryptTests = []struct { } func (s *testEvaluatorSuite) TestSQLDecode(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Decode] for _, tt := range cryptTests { str := types.NewDatum(tt.origin) @@ -63,7 +61,6 @@ func (s *testEvaluatorSuite) TestSQLDecode(c *C) { } func (s *testEvaluatorSuite) TestSQLEncode(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Encode] for _, test := range cryptTests { password := types.NewDatum(test.password) @@ -118,7 +115,6 @@ var aesTests = []struct { } func (s *testEvaluatorSuite) TestAESEncrypt(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.AesEncrypt] for _, tt := range aesTests { variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.BlockEncryptionMode, types.NewDatum(tt.mode)) @@ -138,7 +134,6 @@ func (s *testEvaluatorSuite) TestAESEncrypt(c *C) { } func (s *testEvaluatorSuite) TestAESDecrypt(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.AesDecrypt] for _, tt := range aesTests { variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.BlockEncryptionMode, types.NewDatum(tt.mode)) @@ -230,7 +225,6 @@ var sha1Tests = []struct { } func (s *testEvaluatorSuite) TestSha1Hash(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.SHA] for _, tt := range sha1Tests { in := types.NewDatum(tt.origin) @@ -271,7 +265,6 @@ var sha2Tests = []struct { } func (s *testEvaluatorSuite) TestSha2Hash(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.SHA2] for _, tt := range sha2Tests { str := types.NewDatum(tt.origin) @@ -291,8 +284,6 @@ func (s *testEvaluatorSuite) TestSha2Hash(c *C) { } func (s *testEvaluatorSuite) TestMD5Hash(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args interface{} expected string @@ -329,7 +320,6 @@ func (s *testEvaluatorSuite) TestMD5Hash(c *C) { } func (s *testEvaluatorSuite) TestRandomBytes(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.RandomBytes] f, err := fc.GetFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewDatum(32)})) c.Assert(err, IsNil) @@ -366,7 +356,6 @@ func decodeHex(str string) []byte { } func (s *testEvaluatorSuite) TestCompress(c *C) { - defer testleak.AfterTest(c)() tests := []struct { in interface{} expect interface{} @@ -388,7 +377,6 @@ func (s *testEvaluatorSuite) TestCompress(c *C) { } func (s *testEvaluatorSuite) TestUncompress(c *C) { - defer testleak.AfterTest(c)() tests := []struct { in interface{} expect interface{} @@ -419,7 +407,6 @@ func (s *testEvaluatorSuite) TestUncompress(c *C) { } func (s *testEvaluatorSuite) TestUncompressLength(c *C) { - defer testleak.AfterTest(c)() tests := []struct { in interface{} expect interface{} @@ -449,7 +436,6 @@ func (s *testEvaluatorSuite) TestUncompressLength(c *C) { } func (s *testEvaluatorSuite) TestPassword(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected string diff --git a/expression/builtin_info_test.go b/expression/builtin_info_test.go index 9da4ba9316811..908c1c0875947 100644 --- a/expression/builtin_info_test.go +++ b/expression/builtin_info_test.go @@ -26,11 +26,9 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/printer" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestDatabase(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Database] ctx := mock.NewContext() f, err := fc.GetFunction(ctx, nil) @@ -56,7 +54,6 @@ func (s *testEvaluatorSuite) TestDatabase(c *C) { } func (s *testEvaluatorSuite) TestFoundRows(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() sessionVars.LastFoundRows = 2 @@ -70,7 +67,6 @@ func (s *testEvaluatorSuite) TestFoundRows(c *C) { } func (s *testEvaluatorSuite) TestUser(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() sessionVars.User = &auth.UserIdentity{Username: "root", Hostname: "localhost"} @@ -85,7 +81,6 @@ func (s *testEvaluatorSuite) TestUser(c *C) { } func (s *testEvaluatorSuite) TestCurrentUser(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() sessionVars.User = &auth.UserIdentity{Username: "root", Hostname: "localhost", AuthUsername: "root", AuthHostname: "localhost"} @@ -100,7 +95,6 @@ func (s *testEvaluatorSuite) TestCurrentUser(c *C) { } func (s *testEvaluatorSuite) TestCurrentRole(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() sessionVars.ActiveRoles = make([]*auth.RoleIdentity, 0, 10) @@ -117,7 +111,6 @@ func (s *testEvaluatorSuite) TestCurrentRole(c *C) { } func (s *testEvaluatorSuite) TestConnectionID(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() sessionVars.ConnectionID = uint64(1) @@ -132,7 +125,6 @@ func (s *testEvaluatorSuite) TestConnectionID(c *C) { } func (s *testEvaluatorSuite) TestVersion(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Version] f, err := fc.GetFunction(s.ctx, nil) c.Assert(err, IsNil) @@ -143,8 +135,6 @@ func (s *testEvaluatorSuite) TestVersion(c *C) { } func (s *testEvaluatorSuite) TestBenchMark(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { LoopCount int Expression interface{} @@ -181,7 +171,6 @@ func (s *testEvaluatorSuite) TestBenchMark(c *C) { } func (s *testEvaluatorSuite) TestCharset(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Charset] f, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(nil))) c.Assert(f, IsNil) @@ -189,7 +178,6 @@ func (s *testEvaluatorSuite) TestCharset(c *C) { } func (s *testEvaluatorSuite) TestCoercibility(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Coercibility] f, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(nil))) c.Assert(f, IsNil) @@ -197,7 +185,6 @@ func (s *testEvaluatorSuite) TestCoercibility(c *C) { } func (s *testEvaluatorSuite) TestCollation(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Collation] f, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(nil))) c.Assert(f, IsNil) @@ -205,7 +192,6 @@ func (s *testEvaluatorSuite) TestCollation(c *C) { } func (s *testEvaluatorSuite) TestRowCount(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() sessionVars.StmtCtx.PrevAffectedRows = 10 @@ -225,7 +211,6 @@ func (s *testEvaluatorSuite) TestRowCount(c *C) { // TestTiDBVersion for tidb_server(). func (s *testEvaluatorSuite) TestTiDBVersion(c *C) { - defer testleak.AfterTest(c)() f, err := newFunctionForTest(s.ctx, ast.TiDBVersion, s.primitiveValsToConstants([]interface{}{})...) c.Assert(err, IsNil) v, err := f.Eval(chunk.Row{}) @@ -234,8 +219,6 @@ func (s *testEvaluatorSuite) TestTiDBVersion(c *C) { } func (s *testEvaluatorSuite) TestLastInsertID(c *C) { - defer testleak.AfterTest(c)() - maxUint64 := uint64(math.MaxUint64) cases := []struct { insertID uint64 diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index d0bbf49541315..23f5880eb222a 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -20,12 +20,10 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) func (s *testEvaluatorSuite) TestJSONType(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONType] tbl := []struct { Input interface{} @@ -50,7 +48,6 @@ func (s *testEvaluatorSuite) TestJSONType(c *C) { } func (s *testEvaluatorSuite) TestJSONQuote(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONQuote] tbl := []struct { Input interface{} @@ -79,7 +76,6 @@ func (s *testEvaluatorSuite) TestJSONQuote(c *C) { } func (s *testEvaluatorSuite) TestJSONUnquote(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONUnquote] tbl := []struct { Input interface{} @@ -111,7 +107,6 @@ func (s *testEvaluatorSuite) TestJSONUnquote(c *C) { } func (s *testEvaluatorSuite) TestJSONExtract(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONExtract] jstr := `{"a": [{"aa": [{"aaa": 1}]}], "aaa": 2}` tbl := []struct { @@ -149,7 +144,6 @@ func (s *testEvaluatorSuite) TestJSONExtract(c *C) { // TestJSONSetInsertReplace tests grammar of json_{set,insert,replace}. func (s *testEvaluatorSuite) TestJSONSetInsertReplace(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { fc FunctionClass Input []interface{} @@ -195,7 +189,6 @@ func (s *testEvaluatorSuite) TestJSONSetInsertReplace(c *C) { } func (s *testEvaluatorSuite) TestJSONMerge(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONMerge] tbl := []struct { Input []interface{} @@ -226,7 +219,6 @@ func (s *testEvaluatorSuite) TestJSONMerge(c *C) { } func (s *testEvaluatorSuite) TestJSONMergePreserve(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONMergePreserve] tbl := []struct { Input []interface{} @@ -257,7 +249,6 @@ func (s *testEvaluatorSuite) TestJSONMergePreserve(c *C) { } func (s *testEvaluatorSuite) TestJSONArray(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONArray] tbl := []struct { Input []interface{} @@ -282,7 +273,6 @@ func (s *testEvaluatorSuite) TestJSONArray(c *C) { } func (s *testEvaluatorSuite) TestJSONObject(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONObject] tbl := []struct { Input []interface{} @@ -326,7 +316,6 @@ func (s *testEvaluatorSuite) TestJSONObject(c *C) { } func (s *testEvaluatorSuite) TestJSONRemove(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONRemove] tbl := []struct { Input []interface{} @@ -376,7 +365,6 @@ func (s *testEvaluatorSuite) TestJSONRemove(c *C) { } func (s *testEvaluatorSuite) TestJSONContains(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONContains] tbl := []struct { input []interface{} @@ -456,7 +444,6 @@ func (s *testEvaluatorSuite) TestJSONContains(c *C) { } func (s *testEvaluatorSuite) TestJSONContainsPath(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONContainsPath] jsonString := `{"a": 1, "b": 2, "c": {"d": 4}}` invalidJSON := `{"a": 1` @@ -515,7 +502,6 @@ func (s *testEvaluatorSuite) TestJSONContainsPath(c *C) { } func (s *testEvaluatorSuite) TestJSONLength(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONLength] tbl := []struct { input []interface{} @@ -587,7 +573,6 @@ func (s *testEvaluatorSuite) TestJSONLength(c *C) { } func (s *testEvaluatorSuite) TestJSONKeys(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONKeys] tbl := []struct { input []interface{} @@ -657,7 +642,6 @@ func (s *testEvaluatorSuite) TestJSONKeys(c *C) { } func (s *testEvaluatorSuite) TestJSONDepth(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONDepth] tbl := []struct { input []interface{} @@ -716,7 +700,6 @@ func (s *testEvaluatorSuite) TestJSONDepth(c *C) { } func (s *testEvaluatorSuite) TestJSONArrayAppend(c *C) { - defer testleak.AfterTest(c)() sampleJSON, err := json.ParseBinaryFromString(`{"b": 2}`) c.Assert(err, IsNil) fc := funcs[ast.JSONArrayAppend] @@ -795,7 +778,6 @@ func (s *testEvaluatorSuite) TestJSONArrayAppend(c *C) { } func (s *testEvaluatorSuite) TestJSONSearch(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONSearch] jsonString := `["abc", [{"k": "10"}, "def"], {"x":"abc"}, {"y":"bcd"}]` jsonString2 := `["abc", [{"k": "10"}, "def"], {"x":"ab%d"}, {"y":"abcd"}]` @@ -869,7 +851,6 @@ func (s *testEvaluatorSuite) TestJSONSearch(c *C) { } func (s *testEvaluatorSuite) TestJSONArrayInsert(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONArrayInsert] tbl := []struct { input []interface{} @@ -942,7 +923,6 @@ func (s *testEvaluatorSuite) TestJSONArrayInsert(c *C) { } func (s *testEvaluatorSuite) TestJSONValid(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.JSONValid] tbl := []struct { Input interface{} diff --git a/expression/builtin_json_vec.go b/expression/builtin_json_vec.go index 78842a5ae2648..29391b44595fc 100644 --- a/expression/builtin_json_vec.go +++ b/expression/builtin_json_vec.go @@ -15,6 +15,7 @@ package expression import ( "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" @@ -154,11 +155,77 @@ func (b *builtinJSONSetSig) vecEvalJSON(input *chunk.Chunk, result *chunk.Column } func (b *builtinJSONObjectSig) vectorized() bool { - return false + return true } func (b *builtinJSONObjectSig) vecEvalJSON(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + nr := input.NumRows() + if len(b.args)&1 == 1 { + err := ErrIncorrectParameterCount.GenWithStackByArgs(ast.JSONObject) + return err + } + + jsons := make([]map[string]interface{}, nr) + for i := 0; i < nr; i++ { + jsons[i] = make(map[string]interface{}, len(b.args)>>1) + } + + argBuffers := make([]*chunk.Column, len(b.args)) + var err error + for i := 0; i < len(b.args); i++ { + if i&1 == 0 { + if argBuffers[i], err = b.bufAllocator.get(types.ETString, nr); err != nil { + return err + } + defer func(buf *chunk.Column) { + b.bufAllocator.put(buf) + }(argBuffers[i]) + + if err = b.args[i].VecEvalString(b.ctx, input, argBuffers[i]); err != nil { + return err + } + } else { + if argBuffers[i], err = b.bufAllocator.get(types.ETJson, nr); err != nil { + return err + } + defer func(buf *chunk.Column) { + b.bufAllocator.put(buf) + }(argBuffers[i]) + + if err = b.args[i].VecEvalJSON(b.ctx, input, argBuffers[i]); err != nil { + return err + } + } + } + + result.ReserveJSON(nr) + for i := 0; i < len(b.args); i++ { + if i&1 == 1 { + keyCol := argBuffers[i-1] + valueCol := argBuffers[i] + + var key string + var value json.BinaryJSON + for j := 0; j < nr; j++ { + if keyCol.IsNull(j) { + err := errors.New("JSON documents may not contain NULL member names") + return err + } + key = keyCol.GetString(j) + if valueCol.IsNull(j) { + value = json.CreateBinary(nil) + } else { + value = valueCol.GetJSON(j) + } + jsons[j][key] = value + } + } + } + + for i := 0; i < nr; i++ { + result.AppendJSON(json.CreateBinary(jsons[i])) + } + return nil } func (b *builtinJSONArrayInsertSig) vectorized() bool { @@ -252,9 +319,31 @@ func (b *builtinJSONArrayAppendSig) vecEvalJSON(input *chunk.Chunk, result *chun } func (b *builtinJSONUnquoteSig) vectorized() bool { - return false + return true } func (b *builtinJSONUnquoteSig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + buf, err := b.bufAllocator.get(types.ETJson, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalJSON(b.ctx, input, buf); err != nil { + return err + } + + result.ReserveString(n) + for i := 0; i < n; i++ { + if buf.IsNull(i) { + result.AppendNull() + continue + } + res, err := buf.GetJSON(i).Unquote() + if err != nil { + return err + } + result.AppendString(res) + } + return nil } diff --git a/expression/builtin_json_vec_test.go b/expression/builtin_json_vec_test.go index 9cddca85aa8da..db3a26d4a5d65 100644 --- a/expression/builtin_json_vec_test.go +++ b/expression/builtin_json_vec_test.go @@ -33,15 +33,49 @@ var vecBuiltinJSONCases = map[string][]vecExprBenchCase{ ast.JSONArray: {}, ast.JSONArrayInsert: {}, ast.JSONContains: {}, - ast.JSONObject: {}, - ast.JSONSet: {}, - ast.JSONSearch: {}, - ast.JSONReplace: {}, - ast.JSONDepth: {{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETJson}}}, - ast.JSONUnquote: {}, - ast.JSONRemove: {}, - ast.JSONMerge: {}, - ast.JSONInsert: {}, + ast.JSONObject: { + { + retEvalType: types.ETJson, + childrenTypes: []types.EvalType{ + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + types.ETString, types.ETJson, + }, + geners: []dataGenerator{ + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + &randLenStrGener{10, 20}, nil, + }, + }, + }, + ast.JSONSet: {}, + ast.JSONSearch: {}, + ast.JSONReplace: {}, + ast.JSONDepth: {{retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETJson}}}, + ast.JSONUnquote: { + {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}, geners: []dataGenerator{&jsonStringGener{}}}, + }, + ast.JSONRemove: {}, + ast.JSONMerge: {}, + ast.JSONInsert: {}, ast.JSONQuote: { {retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETJson}}, }, diff --git a/expression/builtin_math_test.go b/expression/builtin_math_test.go index 5a6b449a8da19..962eecbddac1d 100644 --- a/expression/builtin_math_test.go +++ b/expression/builtin_math_test.go @@ -25,12 +25,10 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) func (s *testEvaluatorSuite) TestAbs(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Arg interface{} Ret interface{} @@ -56,8 +54,6 @@ func (s *testEvaluatorSuite) TestAbs(c *C) { } func (s *testEvaluatorSuite) TestCeil(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx tmpIT := sc.IgnoreTruncate sc.IgnoreTruncate = true @@ -122,8 +118,6 @@ func (s *testEvaluatorSuite) TestCeil(c *C) { } func (s *testEvaluatorSuite) TestExp(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect float64 @@ -170,8 +164,6 @@ func (s *testEvaluatorSuite) TestExp(c *C) { } func (s *testEvaluatorSuite) TestFloor(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx tmpIT := sc.IgnoreTruncate sc.IgnoreTruncate = true @@ -244,8 +236,6 @@ func (s *testEvaluatorSuite) TestFloor(c *C) { } func (s *testEvaluatorSuite) TestLog(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args []interface{} expect float64 @@ -286,8 +276,6 @@ func (s *testEvaluatorSuite) TestLog(c *C) { } func (s *testEvaluatorSuite) TestLog2(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect float64 @@ -324,8 +312,6 @@ func (s *testEvaluatorSuite) TestLog2(c *C) { } func (s *testEvaluatorSuite) TestLog10(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect float64 @@ -362,7 +348,6 @@ func (s *testEvaluatorSuite) TestLog10(c *C) { } func (s *testEvaluatorSuite) TestRand(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Rand] f, err := fc.GetFunction(s.ctx, nil) c.Assert(err, IsNil) @@ -383,7 +368,6 @@ func (s *testEvaluatorSuite) TestRand(c *C) { } func (s *testEvaluatorSuite) TestPow(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Arg []interface{} Ret float64 @@ -424,7 +408,6 @@ func (s *testEvaluatorSuite) TestPow(c *C) { } func (s *testEvaluatorSuite) TestRound(c *C) { - defer testleak.AfterTest(c)() newDec := types.NewDecFromStringForTest tbl := []struct { Arg []interface{} @@ -460,7 +443,6 @@ func (s *testEvaluatorSuite) TestRound(c *C) { } func (s *testEvaluatorSuite) TestTruncate(c *C) { - defer testleak.AfterTest(c)() newDec := types.NewDecFromStringForTest tbl := []struct { Arg []interface{} @@ -504,7 +486,6 @@ func (s *testEvaluatorSuite) TestTruncate(c *C) { } func (s *testEvaluatorSuite) TestCRC32(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Arg []interface{} Ret interface{} @@ -531,7 +512,6 @@ func (s *testEvaluatorSuite) TestCRC32(c *C) { } func (s *testEvaluatorSuite) TestConv(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} expected interface{} @@ -595,8 +575,6 @@ func (s *testEvaluatorSuite) TestConv(c *C) { } func (s *testEvaluatorSuite) TestSign(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx tmpIT := sc.IgnoreTruncate sc.IgnoreTruncate = true @@ -631,7 +609,6 @@ func (s *testEvaluatorSuite) TestSign(c *C) { } func (s *testEvaluatorSuite) TestDegrees(c *C) { - defer testleak.AfterTest(c)() sc := s.ctx.GetSessionVars().StmtCtx sc.IgnoreTruncate = false cases := []struct { @@ -672,7 +649,6 @@ func (s *testEvaluatorSuite) TestDegrees(c *C) { } func (s *testEvaluatorSuite) TestSqrt(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Arg []interface{} Ret interface{} @@ -706,7 +682,6 @@ func (s *testEvaluatorSuite) TestPi(c *C) { } func (s *testEvaluatorSuite) TestRadians(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Arg interface{} Ret interface{} @@ -738,7 +713,6 @@ func (s *testEvaluatorSuite) TestRadians(c *C) { } func (s *testEvaluatorSuite) TestSin(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected float64 @@ -780,7 +754,6 @@ func (s *testEvaluatorSuite) TestSin(c *C) { } func (s *testEvaluatorSuite) TestCos(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected float64 @@ -819,8 +792,6 @@ func (s *testEvaluatorSuite) TestCos(c *C) { } func (s *testEvaluatorSuite) TestAcos(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect float64 @@ -857,8 +828,6 @@ func (s *testEvaluatorSuite) TestAcos(c *C) { } func (s *testEvaluatorSuite) TestAsin(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect float64 @@ -895,8 +864,6 @@ func (s *testEvaluatorSuite) TestAsin(c *C) { } func (s *testEvaluatorSuite) TestAtan(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args []interface{} expect float64 @@ -933,7 +900,6 @@ func (s *testEvaluatorSuite) TestAtan(c *C) { } func (s *testEvaluatorSuite) TestTan(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected float64 @@ -971,8 +937,6 @@ func (s *testEvaluatorSuite) TestTan(c *C) { } func (s *testEvaluatorSuite) TestCot(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect float64 diff --git a/expression/builtin_miscellaneous_test.go b/expression/builtin_miscellaneous_test.go index 7a8243d29ccd0..35fcf595e6228 100644 --- a/expression/builtin_miscellaneous_test.go +++ b/expression/builtin_miscellaneous_test.go @@ -22,12 +22,10 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) func (s *testEvaluatorSuite) TestInetAton(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Input interface{} Expected interface{} @@ -93,8 +91,6 @@ func (s *testEvaluatorSuite) TestIsIPv4(c *C) { } func (s *testEvaluatorSuite) TestUUID(c *C) { - defer testleak.AfterTest(c)() - f, err := newFunctionForTest(s.ctx, ast.UUID) c.Assert(err, IsNil) d, err := f.Eval(chunk.Row{}) @@ -120,8 +116,6 @@ func (s *testEvaluatorSuite) TestUUID(c *C) { } func (s *testEvaluatorSuite) TestAnyValue(c *C) { - defer testleak.AfterTest(c)() - tbl := []struct { arg interface{} ret interface{} @@ -324,7 +318,6 @@ func (s *testEvaluatorSuite) TestIsIPv4Compat(c *C) { } func (s *testEvaluatorSuite) TestNameConst(c *C) { - defer testleak.AfterTest(c)() dec := types.NewDecFromFloatForTest(123.123) tm := types.Time{Time: types.FromGoTime(time.Now()), Fsp: 6, Type: mysql.TypeDatetime} du := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second), Fsp: types.DefaultFsp} diff --git a/expression/builtin_op_test.go b/expression/builtin_op_test.go index a0df9a8e91e77..d8998f1e9781e 100644 --- a/expression/builtin_op_test.go +++ b/expression/builtin_op_test.go @@ -21,12 +21,10 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) func (s *testEvaluatorSuite) TestUnary(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected interface{} @@ -66,8 +64,6 @@ func (s *testEvaluatorSuite) TestUnary(c *C) { } func (s *testEvaluatorSuite) TestLogicAnd(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { @@ -130,8 +126,6 @@ func (s *testEvaluatorSuite) TestLogicAnd(c *C) { } func (s *testEvaluatorSuite) TestLeftShift(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} expected uint64 @@ -163,8 +157,6 @@ func (s *testEvaluatorSuite) TestLeftShift(c *C) { } func (s *testEvaluatorSuite) TestRightShift(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} expected uint64 @@ -203,8 +195,6 @@ func (s *testEvaluatorSuite) TestRightShift(c *C) { } func (s *testEvaluatorSuite) TestBitXor(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} expected uint64 @@ -243,8 +233,6 @@ func (s *testEvaluatorSuite) TestBitXor(c *C) { } func (s *testEvaluatorSuite) TestBitOr(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { @@ -290,8 +278,6 @@ func (s *testEvaluatorSuite) TestBitOr(c *C) { } func (s *testEvaluatorSuite) TestLogicOr(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { @@ -358,8 +344,6 @@ func (s *testEvaluatorSuite) TestLogicOr(c *C) { } func (s *testEvaluatorSuite) TestBitAnd(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} expected int64 @@ -398,8 +382,6 @@ func (s *testEvaluatorSuite) TestBitAnd(c *C) { } func (s *testEvaluatorSuite) TestBitNeg(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { @@ -445,8 +427,6 @@ func (s *testEvaluatorSuite) TestBitNeg(c *C) { } func (s *testEvaluatorSuite) TestUnaryNot(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { @@ -498,7 +478,6 @@ func (s *testEvaluatorSuite) TestUnaryNot(c *C) { } func (s *testEvaluatorSuite) TestIsTrueOrFalse(c *C) { - defer testleak.AfterTest(c)() sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { @@ -585,8 +564,6 @@ func (s *testEvaluatorSuite) TestIsTrueOrFalse(c *C) { } func (s *testEvaluatorSuite) TestLogicXor(c *C) { - defer testleak.AfterTest(c)() - sc := s.ctx.GetSessionVars().StmtCtx origin := sc.IgnoreTruncate defer func() { diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index 7a9c364e39fc5..98b952b97bbb3 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -25,11 +25,9 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestBitCount(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -75,7 +73,6 @@ func (s *testEvaluatorSuite) TestBitCount(c *C) { } func (s *testEvaluatorSuite) TestInFunc(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.In] decimal1 := types.NewDecFromFloatForTest(123.121) decimal2 := types.NewDecFromFloatForTest(123.122) @@ -130,14 +127,12 @@ func (s *testEvaluatorSuite) TestInFunc(c *C) { } func (s *testEvaluatorSuite) TestRowFunc(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.RowFunc] _, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums([]interface{}{"1", 1.2, true, 120}...))) c.Assert(err, IsNil) } func (s *testEvaluatorSuite) TestSetVar(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.SetVar] testCases := []struct { args []interface{} @@ -166,7 +161,6 @@ func (s *testEvaluatorSuite) TestSetVar(c *C) { } func (s *testEvaluatorSuite) TestGetVar(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.GetVar] sessionVars := []struct { @@ -200,8 +194,6 @@ func (s *testEvaluatorSuite) TestGetVar(c *C) { } func (s *testEvaluatorSuite) TestValues(c *C) { - defer testleak.AfterTest(c)() - origin := s.ctx.GetSessionVars().StmtCtx.InInsertStmt s.ctx.GetSessionVars().StmtCtx.InInsertStmt = false defer func() { @@ -236,8 +228,6 @@ func (s *testEvaluatorSuite) TestValues(c *C) { } func (s *testEvaluatorSuite) TestSetVarFromColumn(c *C) { - defer testleak.AfterTest(c)() - // Construct arguments. argVarName := &Constant{ Value: types.NewStringDatum("a"), diff --git a/expression/builtin_regexp_vec_const_test.go b/expression/builtin_regexp_vec_const_test.go index dc643f23ebc69..b765126e09175 100644 --- a/expression/builtin_regexp_vec_const_test.go +++ b/expression/builtin_regexp_vec_const_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" ) func genVecBuiltinRegexpBenchCaseForConstants() (baseFunc BuiltinFunc, childrenFieldTypes []*types.FieldType, input *chunk.Chunk, output *chunk.Column) { @@ -60,8 +59,6 @@ func genVecBuiltinRegexpBenchCaseForConstants() (baseFunc BuiltinFunc, childrenF } func (s *testEvaluatorSuite) TestVectorizedBuiltinRegexpForConstants(c *C) { - defer testleak.AfterTest(c)() - bf, childrenFieldTypes, input, output := genVecBuiltinRegexpBenchCaseForConstants() err := bf.vecEvalInt(input, output) c.Assert(err, IsNil) diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 0caab47ef040a..4392cbd994c3e 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -29,12 +29,10 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) func (s *testEvaluatorSuite) TestLengthAndOctetLength(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected int64 @@ -78,8 +76,6 @@ func (s *testEvaluatorSuite) TestLengthAndOctetLength(c *C) { } func (s *testEvaluatorSuite) TestASCII(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args interface{} expected int64 @@ -116,7 +112,6 @@ func (s *testEvaluatorSuite) TestASCII(c *C) { } func (s *testEvaluatorSuite) TestConcat(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} isNil bool @@ -219,7 +214,6 @@ func (s *testEvaluatorSuite) TestConcatSig(c *C) { } func (s *testEvaluatorSuite) TestConcatWS(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} isNil bool @@ -341,7 +335,6 @@ func (s *testEvaluatorSuite) TestConcatWSSig(c *C) { } func (s *testEvaluatorSuite) TestLeft(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -391,7 +384,6 @@ func (s *testEvaluatorSuite) TestLeft(c *C) { } func (s *testEvaluatorSuite) TestRight(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -441,7 +433,6 @@ func (s *testEvaluatorSuite) TestRight(c *C) { } func (s *testEvaluatorSuite) TestRepeat(c *C) { - defer testleak.AfterTest(c)() args := []interface{}{"a", int64(2)} fc := funcs[ast.Repeat] f, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(args...))) @@ -539,7 +530,6 @@ func (s *testEvaluatorSuite) TestRepeatSig(c *C) { } func (s *testEvaluatorSuite) TestLower(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} isNil bool @@ -572,7 +562,6 @@ func (s *testEvaluatorSuite) TestLower(c *C) { } func (s *testEvaluatorSuite) TestUpper(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} isNil bool @@ -605,7 +594,6 @@ func (s *testEvaluatorSuite) TestUpper(c *C) { } func (s *testEvaluatorSuite) TestReverse(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Reverse] f, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(nil))) c.Assert(err, IsNil) @@ -636,7 +624,6 @@ func (s *testEvaluatorSuite) TestReverse(c *C) { } func (s *testEvaluatorSuite) TestStrcmp(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} isNil bool @@ -677,8 +664,6 @@ func (s *testEvaluatorSuite) TestStrcmp(c *C) { } func (s *testEvaluatorSuite) TestReplace(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} isNil bool @@ -719,8 +704,6 @@ func (s *testEvaluatorSuite) TestReplace(c *C) { } func (s *testEvaluatorSuite) TestSubstring(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} isNil bool @@ -768,7 +751,6 @@ func (s *testEvaluatorSuite) TestSubstring(c *C) { } func (s *testEvaluatorSuite) TestConvert(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { str interface{} cs string @@ -828,8 +810,6 @@ func (s *testEvaluatorSuite) TestConvert(c *C) { } func (s *testEvaluatorSuite) TestSubstringIndex(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args []interface{} isNil bool @@ -876,7 +856,6 @@ func (s *testEvaluatorSuite) TestSubstringIndex(c *C) { } func (s *testEvaluatorSuite) TestSpace(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -950,7 +929,6 @@ func (s *testEvaluatorSuite) TestSpaceSig(c *C) { func (s *testEvaluatorSuite) TestLocate(c *C) { // 1. Test LOCATE without binary input. - defer testleak.AfterTest(c)() tbl := []struct { Args []interface{} Want interface{} @@ -1021,7 +999,6 @@ func (s *testEvaluatorSuite) TestLocate(c *C) { } func (s *testEvaluatorSuite) TestTrim(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} isNil bool @@ -1074,7 +1051,6 @@ func (s *testEvaluatorSuite) TestTrim(c *C) { } func (s *testEvaluatorSuite) TestLTrim(c *C) { - defer testleak.AfterTest(c)() cases := []struct { arg interface{} isNil bool @@ -1116,7 +1092,6 @@ func (s *testEvaluatorSuite) TestLTrim(c *C) { } func (s *testEvaluatorSuite) TestRTrim(c *C) { - defer testleak.AfterTest(c)() cases := []struct { arg interface{} isNil bool @@ -1156,7 +1131,6 @@ func (s *testEvaluatorSuite) TestRTrim(c *C) { } func (s *testEvaluatorSuite) TestHexFunc(c *C) { - defer testleak.AfterTest(c)() cases := []struct { arg interface{} isNil bool @@ -1200,7 +1174,6 @@ func (s *testEvaluatorSuite) TestHexFunc(c *C) { } func (s *testEvaluatorSuite) TestUnhexFunc(c *C) { - defer testleak.AfterTest(c)() cases := []struct { arg interface{} isNil bool @@ -1240,7 +1213,6 @@ func (s *testEvaluatorSuite) TestUnhexFunc(c *C) { } func (s *testEvaluatorSuite) TestBitLength(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected int64 @@ -1273,7 +1245,6 @@ func (s *testEvaluatorSuite) TestBitLength(c *C) { } func (s *testEvaluatorSuite) TestChar(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -1323,7 +1294,6 @@ func (s *testEvaluatorSuite) TestChar(c *C) { } func (s *testEvaluatorSuite) TestCharLength(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { input interface{} result interface{} @@ -1372,8 +1342,6 @@ func (s *testEvaluatorSuite) TestCharLength(c *C) { } func (s *testEvaluatorSuite) TestFindInSet(c *C) { - defer testleak.AfterTest(c)() - for _, t := range []struct { str interface{} strlst interface{} @@ -1401,7 +1369,6 @@ func (s *testEvaluatorSuite) TestFindInSet(c *C) { } func (s *testEvaluatorSuite) TestField(c *C) { - defer testleak.AfterTest(c)() stmtCtx := s.ctx.GetSessionVars().StmtCtx origin := stmtCtx.IgnoreTruncate stmtCtx.IgnoreTruncate = true @@ -1637,7 +1604,6 @@ func (s *testEvaluatorSuite) TestInsertBinarySig(c *C) { } func (s *testEvaluatorSuite) TestInstr(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Args []interface{} Want interface{} @@ -1682,8 +1648,6 @@ func (s *testEvaluatorSuite) TestInstr(c *C) { } func (s *testEvaluatorSuite) TestMakeSet(c *C) { - defer testleak.AfterTest(c)() - tbl := []struct { argList []interface{} ret interface{} @@ -1709,7 +1673,6 @@ func (s *testEvaluatorSuite) TestMakeSet(c *C) { } func (s *testEvaluatorSuite) TestOct(c *C) { - defer testleak.AfterTest(c)() octTests := []struct { origin interface{} ret string @@ -1758,7 +1721,6 @@ func (s *testEvaluatorSuite) TestOct(c *C) { } func (s *testEvaluatorSuite) TestFormat(c *C) { - defer testleak.AfterTest(c)() formatTests := []struct { number interface{} precision interface{} @@ -2026,8 +1988,6 @@ func (s *testEvaluatorSuite) TestInsert(c *C) { } func (s *testEvaluatorSuite) TestOrd(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args interface{} expected int64 @@ -2068,8 +2028,6 @@ func (s *testEvaluatorSuite) TestOrd(c *C) { } func (s *testEvaluatorSuite) TestElt(c *C) { - defer testleak.AfterTest(c)() - tbl := []struct { argLst []interface{} ret interface{} @@ -2092,8 +2050,6 @@ func (s *testEvaluatorSuite) TestElt(c *C) { } func (s *testEvaluatorSuite) TestExportSet(c *C) { - defer testleak.AfterTest(c)() - estd := []struct { argLst []interface{} res string @@ -2123,8 +2079,6 @@ func (s *testEvaluatorSuite) TestExportSet(c *C) { } func (s *testEvaluatorSuite) TestBin(c *C) { - defer testleak.AfterTest(c)() - tbl := []struct { Input interface{} Expected interface{} @@ -2156,8 +2110,6 @@ func (s *testEvaluatorSuite) TestBin(c *C) { } func (s *testEvaluatorSuite) TestQuote(c *C) { - defer testleak.AfterTest(c)() - tbl := []struct { arg interface{} ret interface{} @@ -2186,8 +2138,6 @@ func (s *testEvaluatorSuite) TestQuote(c *C) { } func (s *testEvaluatorSuite) TestToBase64(c *C) { - defer testleak.AfterTest(c)() - tests := []struct { args interface{} expect string @@ -2325,7 +2275,6 @@ func (s *testEvaluatorSuite) TestToBase64Sig(c *C) { } func (s *testEvaluatorSuite) TestStringRight(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Right] tests := []struct { str interface{} diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index c6900bf0ac204..3925b11607021 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -80,7 +80,6 @@ func (b *builtinRepeatSig) vecEvalString(input *chunk.Chunk, result *chunk.Colum result.ReserveString(n) nums := buf2.Int64s() for i := 0; i < n; i++ { - // TODO: introduce vectorized null-bitmap to speed it up. if buf.IsNull(i) || buf2.IsNull(i) { result.AppendNull() continue @@ -910,8 +909,9 @@ func (b *builtinInsertSig) vecEvalString(input *chunk.Chunk, result *chunk.Colum result.ReserveString(n) i64s1 := buf1.Int64s() i64s2 := buf2.Int64s() + buf1.MergeNulls(buf2) for i := 0; i < n; i++ { - if buf.IsNull(i) || buf1.IsNull(i) || buf2.IsNull(i) || buf3.IsNull(i) { + if buf.IsNull(i) || buf1.IsNull(i) || buf3.IsNull(i) { result.AppendNull() continue } diff --git a/expression/builtin_test.go b/expression/builtin_test.go index 52e20b3fcb1f1..97dfcb2644b61 100644 --- a/expression/builtin_test.go +++ b/expression/builtin_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testleak" ) func evalBuiltinFunc(f BuiltinFunc, row chunk.Row) (d types.Datum, err error) { @@ -99,8 +98,6 @@ func makeDatums(i interface{}) []types.Datum { } func (s *testEvaluatorSuite) TestIsNullFunc(c *C) { - defer testleak.AfterTest(c)() - fc := funcs[ast.IsNull] f, err := fc.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(1))) c.Assert(err, IsNil) @@ -116,8 +113,6 @@ func (s *testEvaluatorSuite) TestIsNullFunc(c *C) { } func (s *testEvaluatorSuite) TestLock(c *C) { - defer testleak.AfterTest(c)() - lock := funcs[ast.GetLock] f, err := lock.GetFunction(s.ctx, s.datumsToConstants(types.MakeDatums(nil, 1))) c.Assert(err, IsNil) diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 58fa2cc37cde5..50bd30eb53308 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -31,13 +31,11 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" ) func (s *testEvaluatorSuite) TestDate(c *C) { - defer testleak.AfterTest(c)() tblDate := []struct { Input interface{} Expect interface{} @@ -364,7 +362,6 @@ func (s *testEvaluatorSuite) TestDate(c *C) { func (s *testEvaluatorSuite) TestMonthName(c *C) { sc := s.ctx.GetSessionVars().StmtCtx sc.IgnoreZeroInDate = true - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected string @@ -400,7 +397,6 @@ func (s *testEvaluatorSuite) TestMonthName(c *C) { func (s *testEvaluatorSuite) TestDayName(c *C) { sc := s.ctx.GetSessionVars().StmtCtx sc.IgnoreZeroInDate = true - defer testleak.AfterTest(c)() cases := []struct { args interface{} expected string @@ -436,7 +432,6 @@ func (s *testEvaluatorSuite) TestDayName(c *C) { } func (s *testEvaluatorSuite) TestDayOfWeek(c *C) { - defer testleak.AfterTest(c)() sc := s.ctx.GetSessionVars().StmtCtx sc.IgnoreZeroInDate = true cases := []struct { @@ -472,7 +467,6 @@ func (s *testEvaluatorSuite) TestDayOfWeek(c *C) { } func (s *testEvaluatorSuite) TestDayOfMonth(c *C) { - defer testleak.AfterTest(c)() sc := s.ctx.GetSessionVars().StmtCtx sc.IgnoreZeroInDate = true cases := []struct { @@ -508,7 +502,6 @@ func (s *testEvaluatorSuite) TestDayOfMonth(c *C) { } func (s *testEvaluatorSuite) TestDayOfYear(c *C) { - defer testleak.AfterTest(c)() sc := s.ctx.GetSessionVars().StmtCtx sc.IgnoreZeroInDate = true cases := []struct { @@ -544,8 +537,6 @@ func (s *testEvaluatorSuite) TestDayOfYear(c *C) { } func (s *testEvaluatorSuite) TestDateFormat(c *C) { - defer testleak.AfterTest(c)() - // Test case for https://github.com/pingcap/tidb/issues/2908 // SELECT DATE_FORMAT(null,'%Y-%M-%D') args := []types.Datum{types.NewDatum(nil), types.NewStringDatum("%Y-%M-%D")} @@ -591,7 +582,6 @@ func (s *testEvaluatorSuite) TestDateFormat(c *C) { } func (s *testEvaluatorSuite) TestClock(c *C) { - defer testleak.AfterTest(c)() // test hour, minute, second, micro second tbl := []struct { @@ -722,8 +712,6 @@ func (s *testEvaluatorSuite) TestClock(c *C) { } func (s *testEvaluatorSuite) TestTime(c *C) { - defer testleak.AfterTest(c)() - cases := []struct { args interface{} expected string @@ -768,8 +756,6 @@ func resetStmtContext(ctx sessionctx.Context) { } func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) { - defer testleak.AfterTest(c)() - gotime := func(t types.Time, l *time.Location) time.Time { tt, err := t.Time.GoTime(l) c.Assert(err, IsNil) @@ -835,7 +821,6 @@ func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) { } func (s *testEvaluatorSuite) TestIsDuration(c *C) { - defer testleak.AfterTest(c) tbl := []struct { Input string expect bool @@ -857,7 +842,6 @@ func (s *testEvaluatorSuite) TestIsDuration(c *C) { } func (s *testEvaluatorSuite) TestAddTimeSig(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Input string InputDuration string @@ -967,7 +951,6 @@ func (s *testEvaluatorSuite) TestAddTimeSig(c *C) { } func (s *testEvaluatorSuite) TestSubTimeSig(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Input string InputDuration string @@ -1060,7 +1043,6 @@ func (s *testEvaluatorSuite) TestSubTimeSig(c *C) { } func (s *testEvaluatorSuite) TestSysDate(c *C) { - defer testleak.AfterTest(c)() fc := funcs[ast.Sysdate] ctx := mock.NewContext() @@ -1142,8 +1124,6 @@ func builtinDateFormat(ctx sessionctx.Context, args []types.Datum) (d types.Datu } func (s *testEvaluatorSuite) TestFromUnixTime(c *C) { - defer testleak.AfterTest(c)() - tbl := []struct { isDecimal bool integralPart int64 @@ -1210,7 +1190,6 @@ func (s *testEvaluatorSuite) TestFromUnixTime(c *C) { } func (s *testEvaluatorSuite) TestCurrentDate(c *C) { - defer testleak.AfterTest(c)() last := time.Now() fc := funcs[ast.CurrentDate] f, err := fc.GetFunction(mock.NewContext(), s.datumsToConstants(nil)) @@ -1223,7 +1202,6 @@ func (s *testEvaluatorSuite) TestCurrentDate(c *C) { } func (s *testEvaluatorSuite) TestCurrentTime(c *C) { - defer testleak.AfterTest(c)() tfStr := "15:04:05" last := time.Now() @@ -1263,8 +1241,6 @@ func (s *testEvaluatorSuite) TestCurrentTime(c *C) { } func (s *testEvaluatorSuite) TestUTCTime(c *C) { - defer testleak.AfterTest(c)() - last := time.Now().UTC() tfStr := "00:00:00" fc := funcs[ast.UTCTime] @@ -1300,7 +1276,6 @@ func (s *testEvaluatorSuite) TestUTCTime(c *C) { } func (s *testEvaluatorSuite) TestUTCDate(c *C) { - defer testleak.AfterTest(c)() last := time.Now().UTC() fc := funcs[ast.UTCDate] f, err := fc.GetFunction(mock.NewContext(), s.datumsToConstants(nil)) @@ -1730,8 +1705,6 @@ func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) { } func (s *testEvaluatorSuite) TestDateArithFuncs(c *C) { - defer testleak.AfterTest(c)() - date := []string{"2016-12-31", "2017-01-01"} fcAdd := funcs[ast.DateAdd] fcSub := funcs[ast.DateSub] @@ -2017,7 +1990,6 @@ func (s *testEvaluatorSuite) TestTimestamp(c *C) { } func (s *testEvaluatorSuite) TestMakeDate(c *C) { - defer testleak.AfterTest(c)() cases := []struct { args []interface{} expected string @@ -2071,7 +2043,6 @@ func (s *testEvaluatorSuite) TestMakeDate(c *C) { } func (s *testEvaluatorSuite) TestMakeTime(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { Args []interface{} Want interface{} @@ -2395,8 +2366,6 @@ func (s *testEvaluatorSuite) TestPeriodAdd(c *C) { } func (s *testEvaluatorSuite) TestTimeFormat(c *C) { - defer testleak.AfterTest(c)() - // SELECT TIME_FORMAT(null,'%H %k %h %I %l') args := []types.Datum{types.NewDatum(nil), types.NewStringDatum(`%H %k %h %I %l`)} fc := funcs[ast.TimeFormat] @@ -2770,7 +2739,6 @@ func (s *testEvaluatorSuite) TestTidbParseTso(c *C) { } func (s *testEvaluatorSuite) TestGetIntervalFromDecimal(c *C) { - defer testleak.AfterTest(c)() du := baseDateArithmitical{} tests := []struct { diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index e7b9ea02db179..f1a265e096a9d 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -612,11 +612,30 @@ func (b *builtinMinuteSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) } func (b *builtinSecondSig) vectorized() bool { - return false + return true } func (b *builtinSecondSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + buf, err := b.bufAllocator.get(types.ETDuration, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err = b.args[0].VecEvalDuration(b.ctx, input, buf); err != nil { + return err + } + + result.ResizeInt64(n, false) + result.MergeNulls(buf) + i64s := result.Int64s() + for i := 0; i < n; i++ { + if result.IsNull(i) { + continue + } + i64s[i] = int64(buf.GetDuration(i, int(types.UnspecifiedFsp)).Second()) + } + return nil } func (b *builtinNowWithoutArgSig) vectorized() bool { diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index 3abefc332bfe1..3ffc3164db8ae 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -35,7 +35,9 @@ var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ ast.Minute: { {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDuration}, geners: []dataGenerator{&rangeDurationGener{0.2}}}, }, - ast.Second: {}, + ast.Second: { + {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETDuration}, geners: []dataGenerator{&rangeDurationGener{0.2}}}, + }, ast.MicroSecond: {}, ast.Now: {}, ast.DayOfWeek: {}, diff --git a/expression/builtin_vectorized_test.go b/expression/builtin_vectorized_test.go index 91395320e5ec0..545beb21f5e28 100644 --- a/expression/builtin_vectorized_test.go +++ b/expression/builtin_vectorized_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" ) type mockVecPlusIntBuiltinFunc struct { @@ -517,7 +516,6 @@ func vecEvalType(f BuiltinFunc, eType types.EvalType, input *chunk.Chunk, result } func (s *testEvaluatorSuite) TestDoubleRow2Vec(c *C) { - defer testleak.AfterTest(c)() eTypes := []types.EvalType{types.ETInt, types.ETReal, types.ETDecimal, types.ETDuration, types.ETString, types.ETDatetime, types.ETJson} for _, eType := range eTypes { rowDouble, input, result, err := genMockRowDouble(eType, false) @@ -542,7 +540,6 @@ func (s *testEvaluatorSuite) TestDoubleRow2Vec(c *C) { } func (s *testEvaluatorSuite) TestDoubleVec2Row(c *C) { - defer testleak.AfterTest(c)() eTypes := []types.EvalType{types.ETInt, types.ETReal, types.ETDecimal, types.ETDuration, types.ETString, types.ETDatetime, types.ETJson} for _, eType := range eTypes { rowDouble, input, result, err := genMockRowDouble(eType, true) diff --git a/expression/column.go b/expression/column.go index e8b964d865fa7..a95e1eb90c585 100644 --- a/expression/column.go +++ b/expression/column.go @@ -476,6 +476,16 @@ func (col *Column) Vectorized() bool { return true } +// ToInfo converts the expression.Column to model.ColumnInfo for casting values, +// beware it doesn't fill all the fields of the model.ColumnInfo. +func (col *Column) ToInfo() *model.ColumnInfo { + return &model.ColumnInfo{ + ID: col.ID, + Name: col.ColName, + FieldType: *col.RetType, + } +} + // Column2Exprs will transfer column slice to expression slice. func Column2Exprs(cols []*Column) []Expression { result := make([]Expression, 0, len(cols)) diff --git a/expression/column_test.go b/expression/column_test.go index 5eb1b80734d76..4e42bd6366083 100644 --- a/expression/column_test.go +++ b/expression/column_test.go @@ -22,12 +22,9 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestColumn(c *C) { - defer testleak.AfterTest(c)() - col := &Column{RetType: types.NewFieldType(mysql.TypeLonglong), UniqueID: 1} c.Assert(col.Equal(nil, col), IsTrue) @@ -98,8 +95,6 @@ func (s *testEvaluatorSuite) TestColumn(c *C) { } func (s *testEvaluatorSuite) TestColumnHashCode(c *C) { - defer testleak.AfterTest(c)() - col1 := &Column{ UniqueID: 12, } @@ -112,8 +107,6 @@ func (s *testEvaluatorSuite) TestColumnHashCode(c *C) { } func (s *testEvaluatorSuite) TestColumn2Expr(c *C) { - defer testleak.AfterTest(c)() - cols := make([]*Column, 0, 5) for i := 0; i < 5; i++ { cols = append(cols, &Column{UniqueID: int64(i)}) @@ -126,8 +119,6 @@ func (s *testEvaluatorSuite) TestColumn2Expr(c *C) { } func (s *testEvaluatorSuite) TestColInfo2Col(c *C) { - defer testleak.AfterTest(c)() - col0, col1 := &Column{ColName: model.NewCIStr("col0")}, &Column{ColName: model.NewCIStr("col1")} cols := []*Column{col0, col1} colInfo := &model.ColumnInfo{Name: model.NewCIStr("col1")} @@ -140,8 +131,6 @@ func (s *testEvaluatorSuite) TestColInfo2Col(c *C) { } func (s *testEvaluatorSuite) TestIndexInfo2Cols(c *C) { - defer testleak.AfterTest(c)() - col0 := &Column{UniqueID: 0, ID: 0, ColName: model.NewCIStr("col0"), RetType: types.NewFieldType(mysql.TypeLonglong)} col1 := &Column{UniqueID: 1, ID: 1, ColName: model.NewCIStr("col1"), RetType: types.NewFieldType(mysql.TypeLonglong)} indexCol0, indexCol1 := &model.IndexColumn{Name: model.NewCIStr("col0")}, &model.IndexColumn{Name: model.NewCIStr("col1")} @@ -167,7 +156,6 @@ func (s *testEvaluatorSuite) TestIndexInfo2Cols(c *C) { } func (s *testEvaluatorSuite) TestColHybird(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() // bit diff --git a/expression/constant_propagation_test.go b/expression/constant_propagation_test.go index bbe9200ec6614..5206264aad9ff 100644 --- a/expression/constant_propagation_test.go +++ b/expression/constant_propagation_test.go @@ -22,15 +22,16 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/util/testutil" ) var _ = Suite(&testSuite{}) type testSuite struct { - store kv.Storage - dom *domain.Domain - ctx sessionctx.Context + store kv.Storage + dom *domain.Domain + ctx sessionctx.Context + testData testutil.TestData } func (s *testSuite) cleanEnv(c *C) { @@ -45,16 +46,17 @@ func (s *testSuite) cleanEnv(c *C) { func (s *testSuite) SetUpSuite(c *C) { var err error - testleak.BeforeTest() s.store, s.dom, err = newStoreWithBootstrap() c.Assert(err, IsNil) s.ctx = mock.NewContext() + s.testData, err = testutil.LoadTestSuiteData("testdata", "expression_suite") + c.Assert(err, IsNil) } func (s *testSuite) TearDownSuite(c *C) { + c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) s.dom.Close() s.store.Close() - testleak.AfterTest(c)() } func (s *testSuite) TestOuterJoinPropConst(c *C) { @@ -64,201 +66,17 @@ func (s *testSuite) TestOuterJoinPropConst(c *C) { tk.MustExec("create table t1(id bigint primary key, a int, b int);") tk.MustExec("create table t2(id bigint primary key, a int, b int);") - // Positive tests. - tk.MustQuery("explain select * from t1 left join t2 on t1.a > t2.a and t1.a = 1;").Check(testkit.Rows( - "HashLeftJoin_6 33233333.33 root CARTESIAN left outer join, inner:TableReader_11, left cond:[eq(Column#2, 1)]", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 3323.33 root data:Selection_10", - " └─Selection_10 3323.33 cop[tikv] gt(1, Column#5)", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a > t2.a where t1.a = 1;").Check(testkit.Rows( - "HashLeftJoin_7 33233.33 root CARTESIAN left outer join, inner:TableReader_13", - "├─TableReader_10 10.00 root data:Selection_9", - "│ └─Selection_9 10.00 cop[tikv] eq(Column#2, 1)", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_13 3323.33 root data:Selection_12", - " └─Selection_12 3323.33 cop[tikv] gt(1, Column#5)", - " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a = t2.a and t1.a > 1;").Check(testkit.Rows( - "HashLeftJoin_6 10000.00 root left outer join, inner:TableReader_11, equal:[eq(Column#2, Column#5)], left cond:[gt(Column#2, 1)]", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 3333.33 root data:Selection_10", - " └─Selection_10 3333.33 cop[tikv] gt(Column#5, 1), not(isnull(Column#5))", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a = t2.a where t1.a > 1;").Check(testkit.Rows( - "HashLeftJoin_7 4166.67 root left outer join, inner:TableReader_13, equal:[eq(Column#2, Column#5)]", - "├─TableReader_10 3333.33 root data:Selection_9", - "│ └─Selection_9 3333.33 cop[tikv] gt(Column#2, 1)", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_13 3333.33 root data:Selection_12", - " └─Selection_12 3333.33 cop[tikv] gt(Column#5, 1), not(isnull(Column#5))", - " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 right join t2 on t1.a > t2.a where t2.a = 1;").Check(testkit.Rows( - "HashRightJoin_7 33333.33 root CARTESIAN right outer join, inner:TableReader_10", - "├─TableReader_10 3333.33 root data:Selection_9", - "│ └─Selection_9 3333.33 cop[tikv] gt(Column#2, 1)", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_13 10.00 root data:Selection_12", - " └─Selection_12 10.00 cop[tikv] eq(Column#5, 1)", - " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 right join t2 on t1.a = t2.a where t2.a > 1;").Check(testkit.Rows( - "HashRightJoin_7 4166.67 root right outer join, inner:TableReader_10, equal:[eq(Column#2, Column#5)]", - "├─TableReader_10 3333.33 root data:Selection_9", - "│ └─Selection_9 3333.33 cop[tikv] gt(Column#2, 1), not(isnull(Column#2))", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_13 3333.33 root data:Selection_12", - " └─Selection_12 3333.33 cop[tikv] gt(Column#5, 1)", - " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 right join t2 on t1.a = t2.a and t2.a > 1;").Check(testkit.Rows( - "HashRightJoin_6 10000.00 root right outer join, inner:TableReader_9, equal:[eq(Column#2, Column#5)], right cond:gt(Column#5, 1)", - "├─TableReader_9 3333.33 root data:Selection_8", - "│ └─Selection_8 3333.33 cop[tikv] gt(Column#2, 1), not(isnull(Column#2))", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 10000.00 root data:TableScan_10", - " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 right join t2 on t1.a > t2.a and t2.a = 1;").Check(testkit.Rows( - "HashRightJoin_6 33333333.33 root CARTESIAN right outer join, inner:TableReader_9, right cond:eq(Column#5, 1)", - "├─TableReader_9 3333.33 root data:Selection_8", - "│ └─Selection_8 3333.33 cop[tikv] gt(Column#2, 1)", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 10000.00 root data:TableScan_10", - " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - // Negative tests. - tk.MustQuery("explain select * from t1 left join t2 on t1.a = t2.a and t2.a > 1;").Check(testkit.Rows( - "HashLeftJoin_6 10000.00 root left outer join, inner:TableReader_11, equal:[eq(Column#2, Column#5)]", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 3333.33 root data:Selection_10", - " └─Selection_10 3333.33 cop[tikv] gt(Column#5, 1), not(isnull(Column#5))", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a > t2.a and t2.a = 1;").Check(testkit.Rows( - "HashLeftJoin_6 100000.00 root CARTESIAN left outer join, inner:TableReader_11, other cond:gt(Column#2, Column#5)", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 10.00 root data:Selection_10", - " └─Selection_10 10.00 cop[tikv] eq(Column#5, 1), not(isnull(Column#5))", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 right join t2 on t1.a > t2.a and t1.a = 1;").Check(testkit.Rows( - "HashRightJoin_6 100000.00 root CARTESIAN right outer join, inner:TableReader_9, other cond:gt(Column#2, Column#5)", - "├─TableReader_9 10.00 root data:Selection_8", - "│ └─Selection_8 10.00 cop[tikv] eq(Column#2, 1), not(isnull(Column#2))", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 10000.00 root data:TableScan_10", - " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 right join t2 on t1.a = t2.a and t1.a > 1;").Check(testkit.Rows( - "HashRightJoin_6 10000.00 root right outer join, inner:TableReader_9, equal:[eq(Column#2, Column#5)]", - "├─TableReader_9 3333.33 root data:Selection_8", - "│ └─Selection_8 3333.33 cop[tikv] gt(Column#2, 1), not(isnull(Column#2))", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 10000.00 root data:TableScan_10", - " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a = t1.b and t1.a > 1;").Check(testkit.Rows( - "HashLeftJoin_6 100000000.00 root CARTESIAN left outer join, inner:TableReader_10, left cond:[eq(Column#2, Column#3) gt(Column#2, 1)]", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_10 10000.00 root data:TableScan_9", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t2.a = t2.b and t2.a > 1;").Check(testkit.Rows( - "HashLeftJoin_6 26666666.67 root CARTESIAN left outer join, inner:TableReader_11", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 2666.67 root data:Selection_10", - " └─Selection_10 2666.67 cop[tikv] eq(Column#5, Column#6), gt(Column#5, 1)", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - // Constant equal condition merge in outer join. - tk.MustQuery("explain select * from t1 left join t2 on true where t1.a = 1 and false;").Check(testkit.Rows( - "TableDual_8 0.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 left join t2 on true where t1.a = 1 and null;").Check(testkit.Rows( - "TableDual_8 0.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 left join t2 on true where t1.a = null;").Check(testkit.Rows( - "TableDual_8 0.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 left join t2 on true where t1.a = 1 and t1.a = 2;").Check(testkit.Rows( - "TableDual_8 0.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 left join t2 on true where t1.a = 1 and t1.a = 1;").Check(testkit.Rows( - "HashLeftJoin_7 80000.00 root CARTESIAN left outer join, inner:TableReader_12", - "├─TableReader_10 10.00 root data:Selection_9", - "│ └─Selection_9 10.00 cop[tikv] eq(Column#2, 1)", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_12 10000.00 root data:TableScan_11", - " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on false;").Check(testkit.Rows( - "HashLeftJoin_6 80000000.00 root CARTESIAN left outer join, inner:TableDual_9", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableDual_9 8000.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 right join t2 on false;").Check(testkit.Rows( - "HashRightJoin_6 80000000.00 root CARTESIAN right outer join, inner:TableDual_7", - "├─TableDual_7 8000.00 root rows:0", - "└─TableReader_9 10000.00 root data:TableScan_8", - " └─TableScan_8 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a = 1 and t1.a = 2;").Check(testkit.Rows( - "HashLeftJoin_6 80000000.00 root CARTESIAN left outer join, inner:TableDual_9", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableDual_9 8000.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 left join t2 on t1.a =1 where t1.a = 2;").Check(testkit.Rows( - "HashLeftJoin_7 80000.00 root CARTESIAN left outer join, inner:TableDual_11", - "├─TableReader_10 10.00 root data:Selection_9", - "│ └─Selection_9 10.00 cop[tikv] eq(Column#2, 2)", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableDual_11 8000.00 root rows:0", - )) - tk.MustQuery("explain select * from t1 left join t2 on t2.a = 1 and t2.a = 2;").Check(testkit.Rows( - "HashLeftJoin_6 10000.00 root CARTESIAN left outer join, inner:TableReader_11", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_11 0.00 root data:Selection_10", - " └─Selection_10 0.00 cop[tikv] eq(Column#5, 1), eq(Column#5, 2)", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - // Constant propagation for DNF in outer join. - tk.MustQuery("explain select * from t1 left join t2 on t1.a = 1 or (t1.a = 2 and t1.a = 3);").Check(testkit.Rows( - "HashLeftJoin_6 100000000.00 root CARTESIAN left outer join, inner:TableReader_10, left cond:[or(eq(Column#2, 1), 0)]", - "├─TableReader_8 10000.00 root data:TableScan_7", - "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_10 10000.00 root data:TableScan_9", - " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - tk.MustQuery("explain select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3);").Check(testkit.Rows( - "HashLeftJoin_7 80000.00 root CARTESIAN left outer join, inner:TableReader_12", - "├─TableReader_10 10.00 root data:Selection_9", - "│ └─Selection_9 10.00 cop[tikv] or(eq(Column#2, 1), 0)", - "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - "└─TableReader_12 10000.00 root data:TableScan_11", - " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) - // Constant propagation over left outer semi join, filter with aux column should not be derived. - tk.MustQuery("explain select * from t1 where t1.b > 1 or t1.b in (select b from t2);").Check(testkit.Rows( - "Projection_7 8000.00 root Column#1, Column#2, Column#3", - "└─Selection_8 8000.00 root or(gt(Column#3, 1), Column#8)", - " └─HashLeftJoin_9 10000.00 root CARTESIAN left outer semi join, inner:TableReader_13, other cond:eq(Column#3, Column#6)", - " ├─TableReader_11 10000.00 root data:TableScan_10", - " │ └─TableScan_10 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", - " └─TableReader_13 10000.00 root data:TableScan_12", - " └─TableScan_12 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", - )) + var input []string + var output []struct { + SQL string + Result []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } } diff --git a/expression/constant_test.go b/expression/constant_test.go index be6a86866f0c1..7bd7f34dbb698 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" ) var _ = Suite(&testExpressionSuite{}) @@ -81,7 +80,6 @@ func newFunction(funcName string, args ...Expression) Expression { } func (*testExpressionSuite) TestConstantPropagation(c *C) { - defer testleak.AfterTest(c)() tests := []struct { solver []PropagateConstantSolver conditions []Expression @@ -208,7 +206,6 @@ func (*testExpressionSuite) TestConstantPropagation(c *C) { } func (*testExpressionSuite) TestConstraintPropagation(c *C) { - defer testleak.AfterTest(c)() col1 := newColumnWithType(1, types.NewFieldType(mysql.TypeDate)) col2 := newColumnWithType(2, types.NewFieldType(mysql.TypeTimestamp)) tests := []struct { @@ -303,7 +300,6 @@ func (*testExpressionSuite) TestConstraintPropagation(c *C) { } func (*testExpressionSuite) TestConstantFolding(c *C) { - defer testleak.AfterTest(c)() tests := []struct { condition Expression result string @@ -340,7 +336,6 @@ func (*testExpressionSuite) TestConstantFolding(c *C) { } func (*testExpressionSuite) TestDeferredExprNullConstantFold(c *C) { - defer testleak.AfterTest(c)() nullConst := &Constant{ Value: types.NewDatum(nil), RetType: types.NewFieldType(mysql.TypeTiny), @@ -368,7 +363,6 @@ func (*testExpressionSuite) TestDeferredExprNullConstantFold(c *C) { } func (*testExpressionSuite) TestDeferredParamNotNull(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() testTime := time.Now() ctx.GetSessionVars().PreparedParams = []types.Datum{ @@ -435,7 +429,6 @@ func (*testExpressionSuite) TestDeferredParamNotNull(c *C) { } func (*testExpressionSuite) TestDeferredExprNotNull(c *C) { - defer testleak.AfterTest(c)() m := &MockExpr{} ctx := mock.NewContext() cst := &Constant{DeferredExpr: m, RetType: newIntFieldType()} diff --git a/expression/evaluator_test.go b/expression/evaluator_test.go index e5e51cb2e814f..619192184a196 100644 --- a/expression/evaluator_test.go +++ b/expression/evaluator_test.go @@ -36,7 +36,11 @@ var _ = SerialSuites(&testEvaluatorSerialSuites{}) var _ = Suite(&testEvaluatorSuite{}) func TestT(t *testing.T) { + testleak.BeforeTest() + defer testleak.AfterTestT(t) + CustomVerboseFlag = true + *CustomParallelSuiteFlag = true TestingT(t) } @@ -62,12 +66,10 @@ func (s *testEvaluatorSuite) TearDownSuite(c *C) { func (s *testEvaluatorSuite) SetUpTest(c *C) { s.ctx.GetSessionVars().PlanColumnID = 0 - testleak.BeforeTest() } func (s *testEvaluatorSuite) TearDownTest(c *C) { s.ctx.GetSessionVars().StmtCtx.SetWarnings(nil) - testleak.AfterTest(c)() } func (s *testEvaluatorSuite) kindToFieldType(kind byte) types.FieldType { @@ -316,7 +318,6 @@ func (s *testEvaluatorSuite) TestBinopLogic(c *C) { } func (s *testEvaluatorSuite) TestBinopBitop(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { lhs interface{} op string @@ -353,7 +354,6 @@ func (s *testEvaluatorSuite) TestBinopBitop(c *C) { } func (s *testEvaluatorSuite) TestBinopNumeric(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { lhs interface{} op string @@ -505,7 +505,6 @@ func (s *testEvaluatorSuite) TestBinopNumeric(c *C) { } func (s *testEvaluatorSuite) TestExtract(c *C) { - defer testleak.AfterTest(c)() str := "2011-11-11 10:10:10.123456" tbl := []struct { Unit string @@ -551,7 +550,6 @@ func (s *testEvaluatorSuite) TestExtract(c *C) { } func (s *testEvaluatorSuite) TestLike(c *C) { - defer testleak.AfterTest(c)() tests := []struct { input string pattern string @@ -575,7 +573,6 @@ func (s *testEvaluatorSuite) TestLike(c *C) { } func (s *testEvaluatorSuite) TestRegexp(c *C) { - defer testleak.AfterTest(c)() tests := []struct { pattern string input string @@ -611,7 +608,6 @@ func (s *testEvaluatorSuite) TestRegexp(c *C) { } func (s *testEvaluatorSuite) TestUnaryOp(c *C) { - defer testleak.AfterTest(c)() tbl := []struct { arg interface{} op string diff --git a/expression/explain.go b/expression/explain.go index 37bcba7f0252d..93eb01cd498f7 100644 --- a/expression/explain.go +++ b/expression/explain.go @@ -37,8 +37,8 @@ func (expr *ScalarFunction) ExplainInfo() string { } // ExplainInfo implements the Expression interface. -func (expr *Column) ExplainInfo() string { - return expr.String() +func (col *Column) ExplainInfo() string { + return col.String() } // ExplainInfo implements the Expression interface. diff --git a/expression/expression_test.go b/expression/expression_test.go index 71bd527088a89..9ff2d06ff4fc7 100644 --- a/expression/expression_test.go +++ b/expression/expression_test.go @@ -22,12 +22,9 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestNewValuesFunc(c *C) { - defer testleak.AfterTest(c)() - res := NewValuesFunc(s.ctx, 0, types.NewFieldType(mysql.TypeLonglong)) c.Assert(res.FuncName.O, Equals, "values") c.Assert(res.RetType.Tp, Equals, mysql.TypeLonglong) @@ -36,7 +33,6 @@ func (s *testEvaluatorSuite) TestNewValuesFunc(c *C) { } func (s *testEvaluatorSuite) TestEvaluateExprWithNull(c *C) { - defer testleak.AfterTest(c)() tblInfo := newTestTableBuilder("").add("col0", mysql.TypeLonglong).add("col1", mysql.TypeLonglong).build() schema := tableInfoToSchemaForTest(tblInfo) col0 := schema.Columns[0] @@ -57,8 +53,6 @@ func (s *testEvaluatorSuite) TestEvaluateExprWithNull(c *C) { } func (s *testEvaluatorSuite) TestConstant(c *C) { - defer testleak.AfterTest(c)() - sc := &stmtctx.StatementContext{TimeZone: time.Local} c.Assert(Zero.IsCorrelated(), IsFalse) c.Assert(Zero.ConstItem(), IsTrue) @@ -87,8 +81,6 @@ func (s *testEvaluatorSuite) TestIsBinaryLiteral(c *C) { } func (s *testEvaluatorSuite) TestConstItem(c *C) { - defer testleak.AfterTest(c)() - sf := newFunction(ast.Rand) c.Assert(sf.ConstItem(), Equals, false) sf = newFunction(ast.UUID) diff --git a/expression/function_traits_test.go b/expression/function_traits_test.go index 11a21da4beb65..e9919de19d2b6 100644 --- a/expression/function_traits_test.go +++ b/expression/function_traits_test.go @@ -16,11 +16,9 @@ package expression import ( . "github.com/pingcap/check" "github.com/pingcap/parser/ast" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestUnfoldableFuncs(c *C) { - defer testleak.AfterTest(c)() _, ok := unFoldableFunctions[ast.Sysdate] c.Assert(ok, IsTrue) } diff --git a/expression/helper_test.go b/expression/helper_test.go index 21db9667f1063..1a3bcae7ee70e 100644 --- a/expression/helper_test.go +++ b/expression/helper_test.go @@ -14,7 +14,7 @@ package expression import ( - driver "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/types/parser_driver" "strings" "time" @@ -25,11 +25,9 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" ) func (s *testExpressionSuite) TestGetTimeValue(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() v, err := GetTimeValue(ctx, "2012-12-12 00:00:00", mysql.TypeTimestamp, types.MinFsp) c.Assert(err, IsNil) @@ -108,7 +106,6 @@ func (s *testExpressionSuite) TestGetTimeValue(c *C) { } func (s *testExpressionSuite) TestIsCurrentTimestampExpr(c *C) { - defer testleak.AfterTest(c)() buildTimestampFuncCallExpr := func(i int64) *ast.FuncCallExpr { var args []ast.ExprNode if i != 0 { @@ -134,7 +131,6 @@ func (s *testExpressionSuite) TestIsCurrentTimestampExpr(c *C) { } func (s *testExpressionSuite) TestCurrentTimestampTimeZone(c *C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() sessionVars := ctx.GetSessionVars() diff --git a/expression/integration_test.go b/expression/integration_test.go index c2e9d97c6447b..2b044e4143042 100755 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -41,11 +41,11 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" ) var _ = Suite(&testIntegrationSuite{}) +var _ = Suite(&testIntegrationSuite2{}) type testIntegrationSuite struct { store kv.Storage @@ -53,6 +53,10 @@ type testIntegrationSuite struct { ctx sessionctx.Context } +type testIntegrationSuite2 struct { + testIntegrationSuite +} + func (s *testIntegrationSuite) cleanEnv(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -65,7 +69,6 @@ func (s *testIntegrationSuite) cleanEnv(c *C) { func (s *testIntegrationSuite) SetUpSuite(c *C) { var err error - testleak.BeforeTest() s.store, s.dom, err = newStoreWithBootstrap() c.Assert(err, IsNil) s.ctx = mock.NewContext() @@ -74,7 +77,6 @@ func (s *testIntegrationSuite) SetUpSuite(c *C) { func (s *testIntegrationSuite) TearDownSuite(c *C) { s.dom.Close() s.store.Close() - testleak.AfterTest(c)() } func (s *testIntegrationSuite) TestFuncREPEAT(c *C) { @@ -240,7 +242,7 @@ func (s *testIntegrationSuite) TestConvertToBit(c *C) { tk.MustQuery("select a+0 from t").Check(testkit.Rows("20090101000000")) } -func (s *testIntegrationSuite) TestMathBuiltin(c *C) { +func (s *testIntegrationSuite2) TestMathBuiltin(c *C) { ctx := context.Background() defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) @@ -568,7 +570,7 @@ func (s *testIntegrationSuite) TestMathBuiltin(c *C) { tk.MustQuery("select rand(1), rand(2), rand(3)").Check(testkit.Rows("0.6046602879796196 0.16729663442585624 0.7199826688373036")) } -func (s *testIntegrationSuite) TestStringBuiltin(c *C) { +func (s *testIntegrationSuite2) TestStringBuiltin(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -987,7 +989,7 @@ func (s *testIntegrationSuite) TestStringBuiltin(c *C) { "-38.04620119 38.04620115 -38.04620119,38.04620115")) } -func (s *testIntegrationSuite) TestEncryptionBuiltin(c *C) { +func (s *testIntegrationSuite2) TestEncryptionBuiltin(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1152,7 +1154,7 @@ func (s *testIntegrationSuite) TestEncryptionBuiltin(c *C) { result.Check(testkit.Rows("")) } -func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { +func (s *testIntegrationSuite2) TestTimeBuiltin(c *C) { originSQLMode := s.ctx.GetSessionVars().StrictSQLMode s.ctx.GetSessionVars().StrictSQLMode = true defer func() { @@ -2146,7 +2148,7 @@ func (s *testIntegrationSuite) TestDatetimeOverflow(c *C) { tk.MustQuery(`select DATE_SUB('2008-11-23 22:47:31',INTERVAL -266076160 QUARTER);`).Check(testkit.Rows("")) } -func (s *testIntegrationSuite) TestBuiltin(c *C) { +func (s *testIntegrationSuite2) TestBuiltin(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -3501,7 +3503,7 @@ func (s *testIntegrationSuite) TestAggregationBuiltinGroupConcat(c *C) { tk.MustQuery("select * from d").Check(testkit.Rows("hello,h")) } -func (s *testIntegrationSuite) TestOtherBuiltin(c *C) { +func (s *testIntegrationSuite2) TestOtherBuiltin(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/scalar_function_test.go b/expression/scalar_function_test.go index 2b2f84b88e884..b3422375e26b5 100755 --- a/expression/scalar_function_test.go +++ b/expression/scalar_function_test.go @@ -22,12 +22,9 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" ) func (s *testEvaluatorSuite) TestScalarFunction(c *C) { - defer testleak.AfterTest(c)() - a := &Column{ UniqueID: 1, TblName: model.NewCIStr("fei"), @@ -54,7 +51,6 @@ func (s *testEvaluatorSuite) TestScalarFunction(c *C) { } func (s *testEvaluatorSuite) TestScalarFuncs2Exprs(c *C) { - defer testleak.AfterTest(c)() a := &Column{ UniqueID: 1, RetType: types.NewFieldType(mysql.TypeDouble), diff --git a/expression/testdata/expression_suite_in.json b/expression/testdata/expression_suite_in.json new file mode 100644 index 0000000000000..9b94b9cc3c401 --- /dev/null +++ b/expression/testdata/expression_suite_in.json @@ -0,0 +1,39 @@ +[ + { + "name": "TestOuterJoinPropConst", + "cases": [ + // Positive tests. + "explain select * from t1 left join t2 on t1.a > t2.a and t1.a = 1", + "explain select * from t1 left join t2 on t1.a > t2.a where t1.a = 1", + "explain select * from t1 left join t2 on t1.a = t2.a and t1.a > 1", + "explain select * from t1 left join t2 on t1.a = t2.a where t1.a > 1", + "explain select * from t1 right join t2 on t1.a > t2.a where t2.a = 1", + "explain select * from t1 right join t2 on t1.a = t2.a where t2.a > 1", + "explain select * from t1 right join t2 on t1.a = t2.a and t2.a > 1", + "explain select * from t1 right join t2 on t1.a > t2.a and t2.a = 1", + // Negative tests. + "explain select * from t1 left join t2 on t1.a = t2.a and t2.a > 1", + "explain select * from t1 left join t2 on t1.a > t2.a and t2.a = 1", + "explain select * from t1 right join t2 on t1.a > t2.a and t1.a = 1", + "explain select * from t1 right join t2 on t1.a = t2.a and t1.a > 1", + "explain select * from t1 left join t2 on t1.a = t1.b and t1.a > 1", + "explain select * from t1 left join t2 on t2.a = t2.b and t2.a > 1", + // Constant equal condition merge in outer join. + "explain select * from t1 left join t2 on true where t1.a = 1 and false", + "explain select * from t1 left join t2 on true where t1.a = 1 and null", + "explain select * from t1 left join t2 on true where t1.a = null", + "explain select * from t1 left join t2 on true where t1.a = 1 and t1.a = 2", + "explain select * from t1 left join t2 on true where t1.a = 1 and t1.a = 1", + "explain select * from t1 left join t2 on false", + "explain select * from t1 right join t2 on false", + "explain select * from t1 left join t2 on t1.a = 1 and t1.a = 2", + "explain select * from t1 left join t2 on t1.a =1 where t1.a = 2", + "explain select * from t1 left join t2 on t2.a = 1 and t2.a = 2", + // Constant propagation for DNF in outer join. + "explain select * from t1 left join t2 on t1.a = 1 or (t1.a = 2 and t1.a = 3)", + "explain select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3)", + // Constant propagation over left outer semi join, filter with aux column should not be derived. + "explain select * from t1 where t1.b > 1 or t1.b in (select b from t2)" + ] + } +] diff --git a/expression/testdata/expression_suite_out.json b/expression/testdata/expression_suite_out.json new file mode 100644 index 0000000000000..b30569709f9f4 --- /dev/null +++ b/expression/testdata/expression_suite_out.json @@ -0,0 +1,280 @@ +[ + { + "Name": "TestOuterJoinPropConst", + "Cases": [ + { + "SQL": "explain select * from t1 left join t2 on t1.a > t2.a and t1.a = 1", + "Result": [ + "HashLeftJoin_6 33233333.33 root CARTESIAN left outer join, inner:TableReader_11, left cond:[eq(Column#2, 1)]", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 3323.33 root data:Selection_10", + " └─Selection_10 3323.33 cop[tikv] gt(1, Column#5)", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a > t2.a where t1.a = 1", + "Result": [ + "HashLeftJoin_7 33233.33 root CARTESIAN left outer join, inner:TableReader_13", + "├─TableReader_10 10.00 root data:Selection_9", + "│ └─Selection_9 10.00 cop[tikv] eq(Column#2, 1)", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_13 3323.33 root data:Selection_12", + " └─Selection_12 3323.33 cop[tikv] gt(1, Column#5)", + " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = t2.a and t1.a > 1", + "Result": [ + "HashLeftJoin_6 10000.00 root left outer join, inner:TableReader_11, equal:[eq(Column#2, Column#5)], left cond:[gt(Column#2, 1)]", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 3333.33 root data:Selection_10", + " └─Selection_10 3333.33 cop[tikv] gt(Column#5, 1), not(isnull(Column#5))", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = t2.a where t1.a > 1", + "Result": [ + "HashLeftJoin_7 4166.67 root left outer join, inner:TableReader_13, equal:[eq(Column#2, Column#5)]", + "├─TableReader_10 3333.33 root data:Selection_9", + "│ └─Selection_9 3333.33 cop[tikv] gt(Column#2, 1)", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_13 3333.33 root data:Selection_12", + " └─Selection_12 3333.33 cop[tikv] gt(Column#5, 1), not(isnull(Column#5))", + " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on t1.a > t2.a where t2.a = 1", + "Result": [ + "HashRightJoin_7 33333.33 root CARTESIAN right outer join, inner:TableReader_10", + "├─TableReader_10 3333.33 root data:Selection_9", + "│ └─Selection_9 3333.33 cop[tikv] gt(Column#2, 1)", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_13 10.00 root data:Selection_12", + " └─Selection_12 10.00 cop[tikv] eq(Column#5, 1)", + " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on t1.a = t2.a where t2.a > 1", + "Result": [ + "HashRightJoin_7 4166.67 root right outer join, inner:TableReader_10, equal:[eq(Column#2, Column#5)]", + "├─TableReader_10 3333.33 root data:Selection_9", + "│ └─Selection_9 3333.33 cop[tikv] gt(Column#2, 1), not(isnull(Column#2))", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_13 3333.33 root data:Selection_12", + " └─Selection_12 3333.33 cop[tikv] gt(Column#5, 1)", + " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on t1.a = t2.a and t2.a > 1", + "Result": [ + "HashRightJoin_6 10000.00 root right outer join, inner:TableReader_9, equal:[eq(Column#2, Column#5)], right cond:gt(Column#5, 1)", + "├─TableReader_9 3333.33 root data:Selection_8", + "│ └─Selection_8 3333.33 cop[tikv] gt(Column#2, 1), not(isnull(Column#2))", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 10000.00 root data:TableScan_10", + " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on t1.a > t2.a and t2.a = 1", + "Result": [ + "HashRightJoin_6 33333333.33 root CARTESIAN right outer join, inner:TableReader_9, right cond:eq(Column#5, 1)", + "├─TableReader_9 3333.33 root data:Selection_8", + "│ └─Selection_8 3333.33 cop[tikv] gt(Column#2, 1)", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 10000.00 root data:TableScan_10", + " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = t2.a and t2.a > 1", + "Result": [ + "HashLeftJoin_6 10000.00 root left outer join, inner:TableReader_11, equal:[eq(Column#2, Column#5)]", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 3333.33 root data:Selection_10", + " └─Selection_10 3333.33 cop[tikv] gt(Column#5, 1), not(isnull(Column#5))", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a > t2.a and t2.a = 1", + "Result": [ + "HashLeftJoin_6 100000.00 root CARTESIAN left outer join, inner:TableReader_11, other cond:gt(Column#2, Column#5)", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 10.00 root data:Selection_10", + " └─Selection_10 10.00 cop[tikv] eq(Column#5, 1), not(isnull(Column#5))", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on t1.a > t2.a and t1.a = 1", + "Result": [ + "HashRightJoin_6 100000.00 root CARTESIAN right outer join, inner:TableReader_9, other cond:gt(Column#2, Column#5)", + "├─TableReader_9 10.00 root data:Selection_8", + "│ └─Selection_8 10.00 cop[tikv] eq(Column#2, 1), not(isnull(Column#2))", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 10000.00 root data:TableScan_10", + " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on t1.a = t2.a and t1.a > 1", + "Result": [ + "HashRightJoin_6 10000.00 root right outer join, inner:TableReader_9, equal:[eq(Column#2, Column#5)]", + "├─TableReader_9 3333.33 root data:Selection_8", + "│ └─Selection_8 3333.33 cop[tikv] gt(Column#2, 1), not(isnull(Column#2))", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 10000.00 root data:TableScan_10", + " └─TableScan_10 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = t1.b and t1.a > 1", + "Result": [ + "HashLeftJoin_6 100000000.00 root CARTESIAN left outer join, inner:TableReader_10, left cond:[eq(Column#2, Column#3) gt(Column#2, 1)]", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_10 10000.00 root data:TableScan_9", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t2.a = t2.b and t2.a > 1", + "Result": [ + "HashLeftJoin_6 26666666.67 root CARTESIAN left outer join, inner:TableReader_11", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 2666.67 root data:Selection_10", + " └─Selection_10 2666.67 cop[tikv] eq(Column#5, Column#6), gt(Column#5, 1)", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on true where t1.a = 1 and false", + "Result": [ + "TableDual_8 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on true where t1.a = 1 and null", + "Result": [ + "TableDual_8 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on true where t1.a = null", + "Result": [ + "TableDual_8 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on true where t1.a = 1 and t1.a = 2", + "Result": [ + "TableDual_8 0.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on true where t1.a = 1 and t1.a = 1", + "Result": [ + "HashLeftJoin_7 80000.00 root CARTESIAN left outer join, inner:TableReader_12", + "├─TableReader_10 10.00 root data:Selection_9", + "│ └─Selection_9 10.00 cop[tikv] eq(Column#2, 1)", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_12 10000.00 root data:TableScan_11", + " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on false", + "Result": [ + "HashLeftJoin_6 80000000.00 root CARTESIAN left outer join, inner:TableDual_9", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableDual_9 8000.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 right join t2 on false", + "Result": [ + "HashRightJoin_6 80000000.00 root CARTESIAN right outer join, inner:TableDual_7", + "├─TableDual_7 8000.00 root rows:0", + "└─TableReader_9 10000.00 root data:TableScan_8", + " └─TableScan_8 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = 1 and t1.a = 2", + "Result": [ + "HashLeftJoin_6 80000000.00 root CARTESIAN left outer join, inner:TableDual_9", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableDual_9 8000.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a =1 where t1.a = 2", + "Result": [ + "HashLeftJoin_7 80000.00 root CARTESIAN left outer join, inner:TableDual_11", + "├─TableReader_10 10.00 root data:Selection_9", + "│ └─Selection_9 10.00 cop[tikv] eq(Column#2, 2)", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableDual_11 8000.00 root rows:0" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t2.a = 1 and t2.a = 2", + "Result": [ + "HashLeftJoin_6 10000.00 root CARTESIAN left outer join, inner:TableReader_11", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_11 0.00 root data:Selection_10", + " └─Selection_10 0.00 cop[tikv] eq(Column#5, 1), eq(Column#5, 2)", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = 1 or (t1.a = 2 and t1.a = 3)", + "Result": [ + "HashLeftJoin_6 100000000.00 root CARTESIAN left outer join, inner:TableReader_10, left cond:[or(eq(Column#2, 1), 0)]", + "├─TableReader_8 10000.00 root data:TableScan_7", + "│ └─TableScan_7 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_10 10000.00 root data:TableScan_9", + " └─TableScan_9 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3)", + "Result": [ + "HashLeftJoin_7 80000.00 root CARTESIAN left outer join, inner:TableReader_12", + "├─TableReader_10 10.00 root data:Selection_9", + "│ └─Selection_9 10.00 cop[tikv] or(eq(Column#2, 1), 0)", + "│ └─TableScan_8 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_12 10000.00 root data:TableScan_11", + " └─TableScan_11 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select * from t1 where t1.b > 1 or t1.b in (select b from t2)", + "Result": [ + "Projection_7 8000.00 root Column#1, Column#2, Column#3", + "└─Selection_8 8000.00 root or(gt(Column#3, 1), Column#8)", + " └─HashLeftJoin_9 10000.00 root CARTESIAN left outer semi join, inner:TableReader_13, other cond:eq(Column#3, Column#6)", + " ├─TableReader_11 10000.00 root data:TableScan_10", + " │ └─TableScan_10 10000.00 cop[tikv] table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + " └─TableReader_13 10000.00 root data:TableScan_12", + " └─TableScan_12 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo" + ] + } + ] + } +] diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 424c3863852b3..35494969c70ed 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -27,11 +27,10 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) -var _ = Suite(&testInferTypeSuite{}) +var _ = SerialSuites(&testInferTypeSuite{}) type typeInferTestCase struct { sql string @@ -47,12 +46,10 @@ type testInferTypeSuite struct { } func (s *testInferTypeSuite) SetUpSuite(c *C) { - testleak.BeforeTest() s.Parser = parser.New() } func (s *testInferTypeSuite) TearDownSuite(c *C) { - testleak.AfterTest(c)() } func (s *testInferTypeSuite) TestInferType(c *C) { diff --git a/expression/util_test.go b/expression/util_test.go index 3fab967347275..b11c5a9e3dda6 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testleak" ) var _ = check.Suite(&testUtilSuite{}) @@ -254,7 +253,6 @@ func (s testUtilSuite) TestGetStrIntFromConstant(c *check.C) { } func (s *testUtilSuite) TestSubstituteCorCol2Constant(c *check.C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() corCol1 := &CorrelatedColumn{Data: &One.Value} corCol1.RetType = types.NewFieldType(mysql.TypeLonglong) @@ -280,7 +278,6 @@ func (s *testUtilSuite) TestSubstituteCorCol2Constant(c *check.C) { } func (s *testUtilSuite) TestPushDownNot(c *check.C) { - defer testleak.AfterTest(c)() ctx := mock.NewContext() col := &Column{Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)} // !((a=1||a=1)&&a=1) diff --git a/go.sum b/go.sum index 39e926641732e..620db00e4712b 100644 --- a/go.sum +++ b/go.sum @@ -14,7 +14,6 @@ github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mo github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= @@ -64,13 +63,11 @@ github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7a github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20181024230925-c65c006176ff h1:kOkM9whyQYodu09SJ6W3NCsHG7crFaJILQ22Gozp3lg= github.com/golang/groupcache v0.0.0-20181024230925-c65c006176ff/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 h1:ZgQEtGgCBiWRM39fZuwSd1LwSqqSW0hOdXCYYDX0R3I= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.2.0 h1:P3YflyNX/ehuJFLhxviNdFxQPkGK5cDcApsge1SqnvM= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -164,7 +161,6 @@ github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20190822090350-11ea838aedf7/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/kvproto v0.0.0-20190910074005-0e61b6f435c1 h1:DNvxkdcjA0TBIIIF+K2w9KMlTzMZzLZ5JVF26kTCPhg= github.com/pingcap/kvproto v0.0.0-20190910074005-0e61b6f435c1/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= -github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd h1:hWDol43WY5PGhsh3+8794bFHY1bPrmu6bTalpssCrGg= github.com/pingcap/log v0.0.0-20190715063458-479153f07ebd/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9 h1:AJD9pZYm72vMgPcQDww9rkZ1DnWfl0pXV3BOWlkYIjA= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -196,7 +192,6 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d h1:GoAlyOgbOEIFd github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/remyoudompheng/bigfft v0.0.0-20190512091148-babf20351dd7 h1:FUL3b97ZY2EPqg2NbXKuMHs5pXJB9hjj1fDHnF2vl28= github.com/remyoudompheng/bigfft v0.0.0-20190512091148-babf20351dd7/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.18.10+incompatible h1:cy84jW6EVRPa5g9HAHrlbxMSIjBhDSX0OFYyMYminYs= github.com/shirou/gopsutil v2.18.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= @@ -253,20 +248,16 @@ go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf h1:rmttwKPEgG/l4UscTDYtaJgeUsedKPKSyFfNQLI6q+I= go.etcd.io/etcd v0.0.0-20190320044326-77d4b742cdbf/go.mod h1:KSGwdbiFchh5KIC9My2+ZVl5/3ANcwohw50dpPwa2cw= -go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0 h1:cxzIVoETapQEqDhQu3QfnvXAV4AlzcvUCxkVUFw3+EU= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.2.0 h1:6I+W7f5VwC5SV9dNrZ3qXrDB9mD0dyGOi/ZJmYw03T4= go.uber.org/multierr v1.2.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/zap v1.9.1 h1:XCJQEf3W6eZaVwhRBof6ImoYGJSITeKWsyeh3HFu/5o= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.10.0 h1:ORx85nbTijNz8ljznvCMR1ZBIPKFn3jQrag10X2AsuM= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= golang.org/x/crypto v0.0.0-20180608092829-8ac0e0d97ce4/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20180904163835-0709b304e793 h1:u+LnwYTOOW7Ukr/fppxEb1Nwz0AtPflrblfvUudpo+I= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190909091759-094676da4a83/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -289,7 +280,6 @@ golang.org/x/net v0.0.0-20190909003024-a7b16738d86b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 h1:YUO/7uOKsKeq9UokNS62b8FYywz3ker1l1vDZRCRefw= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -300,11 +290,9 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190909082730-f460065e899a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190910064555-bbd175535a8b h1:3S2h5FadpNr0zUUCVZjlKIEYF+KaX/OBplTGo89CYHI= golang.org/x/sys v0.0.0-20190910064555-bbd175535a8b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2 h1:+DCIGbF/swA92ohVg0//6X2IVY3KZs6p9mix0ziNYJM= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -332,7 +320,6 @@ gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4 gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/kv/kv.go b/kv/kv.go index 8413d86f684f9..694434434a482 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -169,7 +169,7 @@ type Transaction interface { // String implements fmt.Stringer interface. String() string // LockKeys tries to lock the entries with the keys in KV store. - LockKeys(ctx context.Context, forUpdateTS uint64, keys ...Key) error + LockKeys(ctx context.Context, killed *uint32, forUpdateTS uint64, keys ...Key) error // SetOption sets an option with a value, when val is nil, uses the default // value of this option. SetOption(opt Option, val interface{}) @@ -187,6 +187,8 @@ type Transaction interface { // SetVars sets variables to the transaction. SetVars(vars *Variables) // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. + // Do not use len(value) == 0 or value == nil to represent non-exist. + // If a key doesn't exist, there shouldn't be any corresponding entry in the result map. BatchGet(ctx context.Context, keys []Key) (map[string][]byte, error) IsPessimistic() bool } diff --git a/kv/mock.go b/kv/mock.go index 70f5ce831d0d7..bd5f6bcc00168 100644 --- a/kv/mock.go +++ b/kv/mock.go @@ -39,7 +39,7 @@ func (t *mockTxn) String() string { return "" } -func (t *mockTxn) LockKeys(_ context.Context, _ uint64, _ ...Key) error { +func (t *mockTxn) LockKeys(_ context.Context, _ *uint32, _ uint64, _ ...Key) error { return nil } diff --git a/kv/mock_test.go b/kv/mock_test.go index 0555feb38f018..18ce2db821d82 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -38,7 +38,7 @@ func (s testMockSuite) TestInterface(c *C) { transaction, err := storage.Begin() c.Check(err, IsNil) - err = transaction.LockKeys(context.Background(), 0, Key("lock")) + err = transaction.LockKeys(context.Background(), nil, 0, Key("lock")) c.Check(err, IsNil) transaction.SetOption(Option(23), struct{}{}) if mock, ok := transaction.(*mockTxn); ok { diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index abd52dde335d9..0b65c84595e13 100755 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -32,7 +32,7 @@ import ( ) const ( - minStep = 1000 + minStep = 30000 maxStep = 2000000 defaultConsumeTime = 10 * time.Second ) diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 9adacbc82d21b..32ba1c49dbb98 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -455,7 +455,7 @@ func (*testSuite) TestNextStep(c *C) { nextStep = autoid.NextStep(678910, 10*time.Second) c.Assert(nextStep, Equals, int64(678910)) nextStep = autoid.NextStep(50000, 10*time.Minute) - c.Assert(nextStep, Equals, int64(1000)) + c.Assert(nextStep, Equals, int64(30000)) } func BenchmarkAllocator_Alloc(b *testing.B) { diff --git a/planner/cascades/optimize.go b/planner/cascades/optimize.go index 0339f4a151472..25e559fbde5c1 100644 --- a/planner/cascades/optimize.go +++ b/planner/cascades/optimize.go @@ -29,7 +29,7 @@ var DefaultOptimizer = NewOptimizer() // Optimizer is the struct for cascades optimizer. type Optimizer struct { - transformationRuleMap map[memo.Operand][]Transformation + transformationRuleMap map[memo.Operand][]TransformationID implementationRuleMap map[memo.Operand][]ImplementationRule } @@ -43,7 +43,7 @@ func NewOptimizer() *Optimizer { } // ResetTransformationRules resets the transformationRuleMap of the optimizer, and returns the optimizer. -func (opt *Optimizer) ResetTransformationRules(rules map[memo.Operand][]Transformation) *Optimizer { +func (opt *Optimizer) ResetTransformationRules(rules map[memo.Operand][]TransformationID) *Optimizer { opt.transformationRuleMap = rules return opt } @@ -54,9 +54,9 @@ func (opt *Optimizer) ResetImplementationRules(rules map[memo.Operand][]Implemen return opt } -// GetTransformationRules gets the all the candidate transformation rules of the optimizer +// GetTransformationIDs gets the all the candidate TransformationIDs of the optimizer // based on the logical plan node. -func (opt *Optimizer) GetTransformationRules(node plannercore.LogicalPlan) []Transformation { +func (opt *Optimizer) GetTransformationIDs(node plannercore.LogicalPlan) []TransformationID { return opt.transformationRuleMap[memo.GetOperand(node)] } @@ -195,8 +195,9 @@ func (opt *Optimizer) exploreGroup(g *memo.Group) error { // findMoreEquiv finds and applies the matched transformation rules. func (opt *Optimizer) findMoreEquiv(g *memo.Group, elem *list.Element) (eraseCur bool, err error) { expr := elem.Value.(*memo.GroupExpr) - for _, rule := range opt.GetTransformationRules(expr.ExprNode) { - pattern := GetPattern(rule) + for _, ruleID := range opt.GetTransformationIDs(expr.ExprNode) { + rule := GetTransformationRule(ruleID) + pattern := GetPattern(ruleID) if !pattern.Operand.Match(memo.GetOperand(expr.ExprNode)) { continue } diff --git a/planner/cascades/stringer_test.go b/planner/cascades/stringer_test.go index 3c204ea2097f3..9e078452051ed 100644 --- a/planner/cascades/stringer_test.go +++ b/planner/cascades/stringer_test.go @@ -51,13 +51,13 @@ func (s *testStringerSuite) TearDownSuite(c *C) { } func (s *testStringerSuite) TestGroupStringer(c *C) { - s.optimizer.ResetTransformationRules(map[memo.Operand][]Transformation{ + s.optimizer.ResetTransformationRules(map[memo.Operand][]TransformationID{ memo.OperandSelection: { - &PushSelDownTableScan{}, - &PushSelDownTableGather{}, + rulePushSelDownTableGather, + rulePushSelDownTableScan, }, memo.OperandDataSource: { - &EnumeratePaths{}, + ruleEnumeratePaths, }, }) defer func() { diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 43b3b93da280e..dad265aca2bd8 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -40,27 +40,51 @@ type Transformation interface { OnTransform(old *memo.ExprIter) (newExprs []*memo.GroupExpr, eraseOld bool, eraseAll bool, err error) } -var defaultTransformationMap = map[memo.Operand][]Transformation{ +// TransformationID is the handle of a Transformation. When we want to add +// a new Transformation rule, we should first add its ID here, and create +// the rule in the transformationRuleList below with the same order. +type TransformationID int + +const ( + rulePushSelDownTableScan TransformationID = iota + rulePushSelDownTableGather + ruleEnumeratePaths +) + +var transformationRuleList = []Transformation{ + &PushSelDownTableScan{}, + &PushSelDownTableGather{}, + &EnumeratePaths{}, +} + +var defaultTransformationMap = map[memo.Operand][]TransformationID{ memo.OperandSelection: { - &PushSelDownTableScan{}, - &PushSelDownTableGather{}, + rulePushSelDownTableScan, + rulePushSelDownTableGather, }, memo.OperandDataSource: { - &EnumeratePaths{}, + ruleEnumeratePaths, }, } -var patternMap = make(map[Transformation]*memo.Pattern) +var patternMap []*memo.Pattern -// GetPattern returns the Pattern of the given Transformation rule. -// It returns the cached Pattern if possible. Otherwise, generate a new Pattern. -func GetPattern(r Transformation) *memo.Pattern { - if p, ok := patternMap[r]; ok { - return p +// init initializes the patternMap when initializing the cascade package. +func init() { + patternMap = make([]*memo.Pattern, len(transformationRuleList)) + for id, rule := range transformationRuleList { + patternMap[id] = rule.GetPattern() } - p := r.GetPattern() - patternMap[r] = p - return p +} + +// GetTransformationRule returns the Transformation rule by its ID. +func GetTransformationRule(id TransformationID) Transformation { + return transformationRuleList[id] +} + +// GetPattern returns the Pattern of the given TransformationID. +func GetPattern(id TransformationID) *memo.Pattern { + return patternMap[id] } // PushSelDownTableScan pushes the selection down to TableScan. diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 76965ff9e81ee..47111c37370d1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2470,6 +2470,7 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model. b.visitInfo = make([]visitInfo, 0) selectLogicalPlan, err := b.Build(ctx, selectNode) if err != nil { + err = ErrViewInvalid.GenWithStackByArgs(dbName.O, tableInfo.Name.O) return nil, err } diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 284db28e05762..f82f50e7dbbed 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -763,15 +763,26 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit allRoles := p.FindAllRole(roles) // Show global grants. var currentPriv mysql.PrivilegeType + var hasGrantOptionPriv bool = false var g string for _, record := range p.User { if record.User == user && record.Host == host { hasGlobalGrant = true + if (record.Privileges & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + currentPriv |= (record.Privileges & ^mysql.GrantPriv) + continue + } currentPriv |= record.Privileges } else { for _, r := range allRoles { if record.User == r.Username && record.Host == r.Hostname { hasGlobalGrant = true + if (record.Privileges & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + currentPriv |= (record.Privileges & ^mysql.GrantPriv) + continue + } currentPriv |= record.Privileges } } @@ -779,13 +790,25 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit } g = userPrivToString(currentPriv) if len(g) > 0 { - s := fmt.Sprintf(`GRANT %s ON *.* TO '%s'@'%s'`, g, user, host) + var s string + if hasGrantOptionPriv { + s = fmt.Sprintf(`GRANT %s ON *.* TO '%s'@'%s' WITH GRANT OPTION`, g, user, host) + + } else { + s = fmt.Sprintf(`GRANT %s ON *.* TO '%s'@'%s'`, g, user, host) + + } gs = append(gs, s) } // This is a mysql convention. if len(gs) == 0 && hasGlobalGrant { - s := fmt.Sprintf("GRANT USAGE ON *.* TO '%s'@'%s'", user, host) + var s string + if hasGrantOptionPriv { + s = fmt.Sprintf("GRANT USAGE ON *.* TO '%s'@'%s' WITH GRANT OPTION", user, host) + } else { + s = fmt.Sprintf("GRANT USAGE ON *.* TO '%s'@'%s'", user, host) + } gs = append(gs, s) } @@ -794,16 +817,36 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit for _, record := range p.DB { if record.User == user && record.Host == host { if _, ok := dbPrivTable[record.DB]; ok { + if (record.Privileges & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + dbPrivTable[record.DB] |= (record.Privileges & ^mysql.GrantPriv) + continue + } dbPrivTable[record.DB] |= record.Privileges } else { + if (record.Privileges & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + dbPrivTable[record.DB] = (record.Privileges & ^mysql.GrantPriv) + continue + } dbPrivTable[record.DB] = record.Privileges } } else { for _, r := range allRoles { if record.User == r.Username && record.Host == r.Hostname { if _, ok := dbPrivTable[record.DB]; ok { + if (record.Privileges & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + dbPrivTable[record.DB] |= (record.Privileges & ^mysql.GrantPriv) + continue + } dbPrivTable[record.DB] |= record.Privileges } else { + if (record.Privileges & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + dbPrivTable[record.DB] = (record.Privileges & ^mysql.GrantPriv) + continue + } dbPrivTable[record.DB] = record.Privileges } } @@ -813,7 +856,14 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit for dbName, priv := range dbPrivTable { g := dbPrivToString(priv) if len(g) > 0 { - s := fmt.Sprintf(`GRANT %s ON %s.* TO '%s'@'%s'`, g, dbName, user, host) + var s string + if hasGrantOptionPriv { + s = fmt.Sprintf(`GRANT %s ON %s.* TO '%s'@'%s' WITH GRANT OPTION`, g, dbName, user, host) + + } else { + s = fmt.Sprintf(`GRANT %s ON %s.* TO '%s'@'%s'`, g, dbName, user, host) + + } gs = append(gs, s) } } @@ -824,16 +874,36 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit recordKey := record.DB + "." + record.TableName if record.User == user && record.Host == host { if _, ok := dbPrivTable[record.DB]; ok { + if (record.TablePriv & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + tablePrivTable[recordKey] |= (record.TablePriv & ^mysql.GrantPriv) + continue + } tablePrivTable[recordKey] |= record.TablePriv } else { + if (record.TablePriv & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + tablePrivTable[recordKey] = (record.TablePriv & ^mysql.GrantPriv) + continue + } tablePrivTable[recordKey] = record.TablePriv } } else { for _, r := range allRoles { if record.User == r.Username && record.Host == r.Hostname { if _, ok := dbPrivTable[record.DB]; ok { + if (record.TablePriv & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + tablePrivTable[recordKey] |= (record.TablePriv & ^mysql.GrantPriv) + continue + } tablePrivTable[recordKey] |= record.TablePriv } else { + if (record.TablePriv & mysql.GrantPriv) > 0 { + hasGrantOptionPriv = true + tablePrivTable[recordKey] = (record.TablePriv & ^mysql.GrantPriv) + continue + } tablePrivTable[recordKey] = record.TablePriv } } @@ -843,7 +913,12 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit for k, priv := range tablePrivTable { g := tablePrivToString(priv) if len(g) > 0 { - s := fmt.Sprintf(`GRANT %s ON %s TO '%s'@'%s'`, g, k, user, host) + var s string + if hasGrantOptionPriv { + s = fmt.Sprintf(`GRANT %s ON %s TO '%s'@'%s' WITH GRANT OPTION`, g, k, user, host) + } else { + s = fmt.Sprintf(`GRANT %s ON %s TO '%s'@'%s'`, g, k, user, host) + } gs = append(gs, s) } } @@ -925,9 +1000,6 @@ func appendUserPrivilegesTableRow(rows [][]types.Datum, user UserRecord) [][]typ guarantee := fmt.Sprintf("'%s'@'%s'", user.User, user.Host) for _, priv := range mysql.AllGlobalPrivs { - if priv == mysql.GrantPriv { - continue - } if user.Privileges&priv > 0 { privilegeType := mysql.Priv2Str[priv] // +---------------------------+---------------+-------------------------+--------------+ diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index bfff0fd7c52fc..b5b843c38014e 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -258,19 +258,32 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { c.Assert(gs[0], Equals, `GRANT Select,Update,Index ON *.* TO 'show'@'localhost'`) // All privileges - AllPrivs := "Select,Insert,Update,Delete,Create,Drop,Process,References,Alter,Show Databases,Super,Execute,Index,Create User,Trigger,Create View,Show View,Create Role,Drop Role,CREATE TEMPORARY TABLES,LOCK TABLES,CREATE ROUTINE,ALTER ROUTINE,EVENT,SHUTDOWN" mustExec(c, se, `GRANT ALL ON *.* TO 'show'@'localhost';`) gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 1) - c.Assert(gs[0], Equals, `GRANT `+AllPrivs+` ON *.* TO 'show'@'localhost'`) + c.Assert(gs[0], Equals, `GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`) + + // All privileges with grant option + mustExec(c, se, `GRANT ALL ON *.* TO 'show'@'localhost' WITH GRANT OPTION;`) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 1) + c.Assert(gs[0], Equals, `GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost' WITH GRANT OPTION`) + + // Revoke grant option + mustExec(c, se, `REVOKE GRANT OPTION ON *.* FROM 'show'@'localhost';`) + gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) + c.Assert(err, IsNil) + c.Assert(gs, HasLen, 1) + c.Assert(gs[0], Equals, `GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`) // Add db scope privileges mustExec(c, se, `GRANT Select ON test.* TO 'show'@'localhost';`) gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 2) - expected := []string{`GRANT ` + AllPrivs + ` ON *.* TO 'show'@'localhost'`, + expected := []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, `GRANT Select ON test.* TO 'show'@'localhost'`} c.Assert(testutil.CompareUnorderedStringSlice(gs, expected), IsTrue) @@ -278,7 +291,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 3) - expected = []string{`GRANT ` + AllPrivs + ` ON *.* TO 'show'@'localhost'`, + expected = []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, `GRANT Select ON test.* TO 'show'@'localhost'`, `GRANT Index ON test1.* TO 'show'@'localhost'`} c.Assert(testutil.CompareUnorderedStringSlice(gs, expected), IsTrue) @@ -287,7 +300,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 3) - expected = []string{`GRANT ` + AllPrivs + ` ON *.* TO 'show'@'localhost'`, + expected = []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, `GRANT Select ON test.* TO 'show'@'localhost'`, `GRANT ALL PRIVILEGES ON test1.* TO 'show'@'localhost'`} c.Assert(testutil.CompareUnorderedStringSlice(gs, expected), IsTrue) @@ -297,7 +310,7 @@ func (s *testPrivilegeSuite) TestShowGrants(c *C) { gs, err = pc.ShowGrants(se, &auth.UserIdentity{Username: "show", Hostname: "localhost"}, nil) c.Assert(err, IsNil) c.Assert(gs, HasLen, 4) - expected = []string{`GRANT ` + AllPrivs + ` ON *.* TO 'show'@'localhost'`, + expected = []string{`GRANT ALL PRIVILEGES ON *.* TO 'show'@'localhost'`, `GRANT Select ON test.* TO 'show'@'localhost'`, `GRANT ALL PRIVILEGES ON test1.* TO 'show'@'localhost'`, `GRANT Update ON test.test TO 'show'@'localhost'`} @@ -493,6 +506,14 @@ func (s *testPrivilegeSuite) TestUseDB(c *C) { mustExec(c, se, "CREATE USER 'usesuper'") mustExec(c, se, "CREATE USER 'usenobody'") mustExec(c, se, "GRANT ALL ON *.* TO 'usesuper'") + //without grant option + c.Assert(se.Auth(&auth.UserIdentity{Username: "usesuper", Hostname: "localhost", AuthUsername: "usesuper", AuthHostname: "%"}, nil, nil), IsTrue) + _, e := se.Execute(context.Background(), "GRANT SELECT ON mysql.* TO 'usenobody'") + c.Assert(e, NotNil) + //with grant option + se = newSession(c, s.store, s.dbName) + // high privileged user + mustExec(c, se, "GRANT ALL ON *.* TO 'usesuper' WITH GRANT OPTION") c.Assert(se.Auth(&auth.UserIdentity{Username: "usesuper", Hostname: "localhost", AuthUsername: "usesuper", AuthHostname: "%"}, nil, nil), IsTrue) mustExec(c, se, "use mysql") // low privileged user @@ -539,7 +560,7 @@ func (s *testPrivilegeSuite) TestSetGlobal(c *C) { func (s *testPrivilegeSuite) TestCreateDropUser(c *C) { se := newSession(c, s.store, s.dbName) mustExec(c, se, `CREATE USER tcd1, tcd2`) - mustExec(c, se, `GRANT ALL ON *.* to tcd2`) + mustExec(c, se, `GRANT ALL ON *.* to tcd2 WITH GRANT OPTION`) // should fail c.Assert(se.Auth(&auth.UserIdentity{Username: "tcd1", Hostname: "localhost", AuthUsername: "tcd1", AuthHostname: "%"}, nil, nil), IsTrue) @@ -582,7 +603,7 @@ func (s *testPrivilegeSuite) TestAnalyzeTable(c *C) { // high privileged user mustExec(c, se, "CREATE USER 'asuper'") mustExec(c, se, "CREATE USER 'anobody'") - mustExec(c, se, "GRANT ALL ON *.* TO 'asuper'") + mustExec(c, se, "GRANT ALL ON *.* TO 'asuper' WITH GRANT OPTION") mustExec(c, se, "CREATE DATABASE atest") mustExec(c, se, "use atest") mustExec(c, se, "CREATE TABLE t1 (a int)") @@ -683,12 +704,12 @@ func (s *testPrivilegeSuite) TestUserTableConsistency(c *C) { var res bytes.Buffer buf.WriteString("select ") i := 0 - for _, priv := range mysql.Priv2UserCol { + for _, priv := range mysql.AllGlobalPrivs { if i != 0 { buf.WriteString(", ") res.WriteString(" ") } - buf.WriteString(priv) + buf.WriteString(mysql.Priv2UserCol[priv]) res.WriteString("Y") i++ } diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 04cb65404236e..4b0c6e5e940ef 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -15,6 +15,8 @@ package session_test import ( "fmt" + "sync" + "sync/atomic" "time" . "github.com/pingcap/check" @@ -392,3 +394,30 @@ func (s *testPessimisticSuite) TestOptimisticConflicts(c *C) { tk.MustExec("commit") tk.MustQuery("select * from conflict").Check(testkit.Rows("1 3")) } + +func (s *testPessimisticSuite) TestWaitLockKill(c *C) { + // Test kill command works on waiting pessimistic lock. + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists test_kill") + tk.MustExec("create table test_kill (id int primary key, c int)") + tk.MustExec("insert test_kill values (1, 1)") + tk.MustExec("begin pessimistic") + tk2.MustExec("begin pessimistic") + tk.MustQuery("select * from test_kill where id = 1 for update") + + var wg sync.WaitGroup + wg.Add(1) + go func() { + time.Sleep(500 * time.Millisecond) + sessVars := tk2.Se.GetSessionVars() + succ := atomic.CompareAndSwapUint32(&sessVars.Killed, 0, 1) + c.Assert(succ, IsTrue) + wg.Wait() + }() + _, err := tk2.Exec("update test_kill set c = c + 1 where id = 1") + wg.Done() + c.Assert(err, NotNil) + c.Assert(terror.ErrorEqual(err, tikv.ErrQueryInterrupted), IsTrue) + tk.MustExec("rollback") +} diff --git a/session/tidb.go b/session/tidb.go index 0805c72399453..37c075ea0aeaa 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -20,6 +20,7 @@ package session import ( "context" "sync" + "sync/atomic" "time" "github.com/opentracing/opentracing-go" @@ -64,8 +65,8 @@ func (dm *domainMap) Get(store kv.Storage) (d *domain.Domain, err error) { return } - ddlLease := schemaLease - statisticLease := statsLease + ddlLease := time.Duration(atomic.LoadInt64(&schemaLease)) + statisticLease := time.Duration(atomic.LoadInt64(&statsLease)) err = util.RunWithRetry(util.DefaultMaxRetries, util.RetryInterval, func() (retry bool, err1 error) { logutil.BgLogger().Info("new domain", zap.String("store", store.UUID()), @@ -111,27 +112,27 @@ var ( // Default schema lease time is 1 second, you can change it with a proper time, // but you must know that too little may cause badly performance degradation. // For production, you should set a big schema lease, like 300s+. - schemaLease = 1 * time.Second + schemaLease = int64(1 * time.Second) // statsLease is the time for reload stats table. - statsLease = 3 * time.Second + statsLease = int64(3 * time.Second) ) // SetSchemaLease changes the default schema lease time for DDL. // This function is very dangerous, don't use it if you really know what you do. // SetSchemaLease only affects not local storage after bootstrapped. func SetSchemaLease(lease time.Duration) { - schemaLease = lease + atomic.StoreInt64(&schemaLease, int64(lease)) } // SetStatsLease changes the default stats lease time for loading stats info. func SetStatsLease(lease time.Duration) { - statsLease = lease + atomic.StoreInt64(&statsLease, int64(lease)) } // DisableStats4Test disables the stats for tests. func DisableStats4Test() { - statsLease = -1 + SetStatsLease(-1) } // Parse parses a query string to raw ast.StmtNode. diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 1361ffdd30d95..3dc2b10edcb90 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -23,6 +23,7 @@ import ( "time" "github.com/golang/protobuf/proto" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" @@ -708,6 +709,12 @@ func (c *RPCClient) checkArgs(ctx context.Context, addr string) (*rpcHandler, er // SendRequest sends a request to mock cluster. func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("RPCClient.SendRequest", opentracing.ChildOf(span.Context())) + defer span1.Finish() + ctx = opentracing.ContextWithSpan(ctx, span1) + } + failpoint.Inject("rpcServerBusy", func(val failpoint.Value) { if val.(bool) { failpoint.Return(tikvrpc.GenRegionErrorResp(req, &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}})) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index c4f6cb7a67420..608b5481af384 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -46,7 +46,7 @@ type twoPhaseCommitAction interface { type actionPrewrite struct{} type actionCommit struct{} type actionCleanup struct{} -type actionPessimisticLock struct{} +type actionPessimisticLock struct{ killed *uint32 } type actionPessimisticRollback struct{} var ( @@ -643,7 +643,7 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { } } -func (actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchKeys) error { +func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchKeys) error { mutations := make([]*pb.Mutation, len(batch.keys)) for i, k := range batch.keys { mut := &pb.Mutation{ @@ -679,7 +679,7 @@ func (actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if err != nil { return errors.Trace(err) } - err = c.pessimisticLockKeys(bo, batch.keys) + err = c.pessimisticLockKeys(bo, action.killed, batch.keys) return errors.Trace(err) } if resp.Resp == nil { @@ -712,11 +712,26 @@ func (actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff } locks = append(locks, lock) } - _, err = c.store.lockResolver.ResolveLocks(bo, c.startTS, locks) + var expire int64 + expire, err = c.store.lockResolver.ResolveLocks(bo, c.startTS, locks) if err != nil { return errors.Trace(err) } - // Because we already waited on tikv, no need to Backoff here. + + if err1 := bo.BackoffWithMaxSleep(BoTxnLock, int(expire), errors.New(locks[0].String())); err1 != nil { + return err1 + } + // Handle the killed flag when waiting for the pessimistic lock. + // When a txn runs into LockKeys() and backoff here, it has no chance to call + // executor.Next() and check the killed flag. + if action.killed != nil { + // Do not reset the killed flag here! + // actionPessimisticLock runs on each region parallelly, we have to consider that + // the error may be dropped. + if atomic.LoadUint32(action.killed) == 1 { + return ErrQueryInterrupted + } + } } } @@ -918,8 +933,8 @@ func (c *twoPhaseCommitter) cleanupKeys(bo *Backoffer, keys [][]byte) error { return c.doActionOnKeys(bo, actionCleanup{}, keys) } -func (c *twoPhaseCommitter) pessimisticLockKeys(bo *Backoffer, keys [][]byte) error { - return c.doActionOnKeys(bo, actionPessimisticLock{}, keys) +func (c *twoPhaseCommitter) pessimisticLockKeys(bo *Backoffer, killed *uint32, keys [][]byte) error { + return c.doActionOnKeys(bo, actionPessimisticLock{killed}, keys) } func (c *twoPhaseCommitter) pessimisticRollbackKeys(bo *Backoffer, keys [][]byte) error { diff --git a/store/tikv/2pc_test.go b/store/tikv/2pc_test.go index 385a8df18493c..14e0284bd341d 100644 --- a/store/tikv/2pc_test.go +++ b/store/tikv/2pc_test.go @@ -506,7 +506,7 @@ func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) { c.Assert(txn.Set(key, key), IsNil) txn.DelOption(kv.PresumeKeyNotExistsError) txn.DelOption(kv.PresumeKeyNotExists) - err := txn.LockKeys(context.Background(), txn.startTS, key) + err := txn.LockKeys(context.Background(), nil, txn.startTS, key) c.Assert(err, NotNil) c.Assert(txn.Delete(key), IsNil) key2 := kv.Key("key2") @@ -518,9 +518,9 @@ func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) { func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) { txn := s.begin(c) txn.SetOption(kv.Pessimistic, true) - err := txn.LockKeys(context.Background(), 100, kv.Key("abc"), kv.Key("def")) + err := txn.LockKeys(context.Background(), nil, 100, kv.Key("abc"), kv.Key("def")) c.Assert(err, IsNil) - err = txn.LockKeys(context.Background(), 100, kv.Key("abc"), kv.Key("def")) + err = txn.LockKeys(context.Background(), nil, 100, kv.Key("abc"), kv.Key("def")) c.Assert(err, IsNil) c.Assert(txn.lockKeys, HasLen, 2) } @@ -530,11 +530,11 @@ func (s *testCommitterSuite) TestPessimisticTTL(c *C) { txn := s.begin(c) txn.SetOption(kv.Pessimistic, true) time.Sleep(time.Millisecond * 100) - err := txn.LockKeys(context.Background(), txn.startTS, key) + err := txn.LockKeys(context.Background(), nil, txn.startTS, key) c.Assert(err, IsNil) time.Sleep(time.Millisecond * 100) key2 := kv.Key("key2") - err = txn.LockKeys(context.Background(), txn.startTS, key2) + err = txn.LockKeys(context.Background(), nil, txn.startTS, key2) c.Assert(err, IsNil) lockInfo := s.getLockInfo(c, key) elapsedTTL := lockInfo.LockTtl - PessimisticLockTTL diff --git a/store/tikv/error.go b/store/tikv/error.go index 5e775218a9f5e..574e460454912 100644 --- a/store/tikv/error.go +++ b/store/tikv/error.go @@ -38,6 +38,7 @@ var ( ErrRegionUnavailable = terror.ClassTiKV.New(mysql.ErrRegionUnavailable, mysql.MySQLErrName[mysql.ErrRegionUnavailable]) ErrTiKVServerBusy = terror.ClassTiKV.New(mysql.ErrTiKVServerBusy, mysql.MySQLErrName[mysql.ErrTiKVServerBusy]) ErrGCTooEarly = terror.ClassTiKV.New(mysql.ErrGCTooEarly, mysql.MySQLErrName[mysql.ErrGCTooEarly]) + ErrQueryInterrupted = terror.ClassTiKV.New(mysql.ErrQueryInterrupted, mysql.MySQLErrName[mysql.ErrQueryInterrupted]) ) // ErrDeadlock wraps *kvrpcpb.Deadlock to implement the error interface. @@ -60,6 +61,7 @@ func init() { mysql.ErrTiKVServerBusy: mysql.ErrTiKVServerBusy, mysql.ErrGCTooEarly: mysql.ErrGCTooEarly, mysql.ErrTruncatedWrongValue: mysql.ErrTruncatedWrongValue, + mysql.ErrQueryInterrupted: mysql.ErrQueryInterrupted, } terror.ErrClassToMySQLCodes[terror.ClassTiKV] = tikvMySQLErrCodes } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 96a30073ca4d1..584232e3e780f 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -21,6 +21,7 @@ import ( "time" "unsafe" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" @@ -62,6 +63,10 @@ type tikvSnapshot struct { // Cache the result of BatchGet. // The invariance is that calling BatchGet multiple times using the same start ts, // the result should not change. + // NOTE: This representation here is different from the BatchGet API. + // cached use len(value)=0 to represent a key-value entry doesn't exist (a reliable truth from TiKV). + // In the BatchGet API, it use no key-value entry to represent non-exist. + // It's OK as long as there are no zero-byte values in the protocol. cached map[string][]byte } @@ -95,7 +100,9 @@ func (s *tikvSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string] tmp := keys[:0] for _, key := range keys { if val, ok := s.cached[string(key)]; ok { - m[string(key)] = val + if len(val) > 0 { + m[string(key)] = val + } } else { tmp = append(tmp, key) } @@ -121,6 +128,7 @@ func (s *tikvSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string] if len(v) == 0 { return } + mu.Lock() m[string(k)] = v mu.Unlock() @@ -138,8 +146,8 @@ func (s *tikvSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string] if s.cached == nil { s.cached = make(map[string][]byte, len(m)) } - for key, value := range m { - s.cached[key] = value + for _, key := range keys { + s.cached[string(key)] = m[string(key)] } return m, nil @@ -253,6 +261,12 @@ func (s *tikvSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, coll // Get gets the value for key k from snapshot. func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("tikvSnapshot.get", opentracing.ChildOf(span.Context())) + defer span1.Finish() + ctx = opentracing.ContextWithSpan(ctx, span1) + } + ctx = context.WithValue(ctx, txnStartKey, s.version.Ver) val, err := s.get(NewBackoffer(ctx, getMaxBackoff), k) if err != nil { @@ -272,6 +286,10 @@ func (s *tikvSnapshot) get(bo *Backoffer, k kv.Key) ([]byte, error) { } } + failpoint.Inject("snapshot-get-cache-fail", func(_ failpoint.Value) { + panic("cache miss") + }) + sender := NewRegionRequestSender(s.store.regionCache, s.store.client) req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, diff --git a/store/tikv/snapshot_test.go b/store/tikv/snapshot_test.go index 7ea31aebe801e..d8ea5792975c2 100644 --- a/store/tikv/snapshot_test.go +++ b/store/tikv/snapshot_test.go @@ -19,6 +19,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/logutil" @@ -117,6 +118,27 @@ func (s *testSnapshotSuite) TestBatchGet(c *C) { } } +func (s *testSnapshotSuite) TestSnapshotCache(c *C) { + txn := s.beginTxn(c) + c.Assert(txn.Set(kv.Key("x"), []byte("x")), IsNil) + c.Assert(txn.Commit(context.Background()), IsNil) + + txn = s.beginTxn(c) + snapshot := newTiKVSnapshot(s.store, kv.Version{Ver: txn.StartTS()}, 0) + _, err := snapshot.BatchGet(context.Background(), []kv.Key{kv.Key("x"), kv.Key("y")}) + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/snapshot-get-cache-fail", `return(true)`), IsNil) + ctx := context.WithValue(context.Background(), "TestSnapshotCache", true) + _, err = snapshot.Get(ctx, kv.Key("x")) + c.Assert(err, IsNil) + + _, err = snapshot.Get(ctx, kv.Key("y")) + c.Assert(kv.IsErrNotFound(err), IsTrue) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/snapshot-get-cache-fail"), IsNil) +} + func (s *testSnapshotSuite) TestBatchGetNotExist(c *C) { for _, rowNum := range s.rowNums { logutil.BgLogger().Debug("test BatchGetNotExist", diff --git a/store/tikv/ticlient_test.go b/store/tikv/ticlient_test.go index e771a9bd61315..49759035ec990 100644 --- a/store/tikv/ticlient_test.go +++ b/store/tikv/ticlient_test.go @@ -119,7 +119,7 @@ func (s *testTiclientSuite) TestSingleKey(c *C) { txn := s.beginTxn(c) err := txn.Set(encodeKey(s.prefix, "key"), []byte("value")) c.Assert(err, IsNil) - err = txn.LockKeys(context.Background(), 0, encodeKey(s.prefix, "key")) + err = txn.LockKeys(context.Background(), nil, 0, encodeKey(s.prefix, "key")) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 123d22aa0e7ee..4761421ba85f0 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "sync" + "sync/atomic" "time" "github.com/dgryski/go-farm" @@ -380,7 +381,7 @@ func (txn *tikvTxn) rollbackPessimisticLocks() error { return txn.committer.pessimisticRollbackKeys(NewBackoffer(context.Background(), cleanupMaxBackoff), txn.lockKeys) } -func (txn *tikvTxn) LockKeys(ctx context.Context, forUpdateTS uint64, keysInput ...kv.Key) error { +func (txn *tikvTxn) LockKeys(ctx context.Context, killed *uint32, forUpdateTS uint64, keysInput ...kv.Key) error { // Exclude keys that are already locked. keys := make([][]byte, 0, len(keysInput)) txn.mu.Lock() @@ -424,7 +425,13 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, forUpdateTS uint64, keysInput // If the number of keys greater than 1, it can be on different region, // concurrently execute on multiple regions may lead to deadlock. txn.committer.isFirstLock = len(txn.lockKeys) == 0 && len(keys) == 1 - err := txn.committer.pessimisticLockKeys(bo, keys) + err := txn.committer.pessimisticLockKeys(bo, killed, keys) + if killed != nil { + // If the kill signal is received during waiting for pessimisticLock, + // pessimisticLockKeys would handle the error but it doesn't reset the flag. + // We need to reset the killed flag here. + atomic.CompareAndSwapUint32(killed, 1, 0) + } if err != nil { for _, key := range keys { txn.us.DeleteKeyExistErrInfo(key) diff --git a/util/chunk/column.go b/util/chunk/column.go index 9f08be625b41b..b8a844153c3bc 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -663,6 +663,14 @@ func (c *Column) CopyReconstruct(sel []int, dst *Column) *Column { // The caller should ensure that all these columns have the same // length, and data stored in the result column is fixed-length type. func (c *Column) MergeNulls(cols ...*Column) { + if !c.isFixed() { + panic("result column should be fixed-length type") + } + for _, col := range cols { + if c.length != col.length { + panic("should ensure all columns have the same length") + } + } for _, col := range cols { for i := range c.nullBitmap { // bit 0 is null, 1 is not null, so do AND operations here.