From e40f8c0881aad4438b8f738ee35ff54dd64205c5 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 12 May 2021 16:40:58 +0800 Subject: [PATCH 01/27] *: fix revoke statement for CURRENT_USER() and refine error message (#24052) * *: fix revoke statement for CURRENT_USER() and refine error message --- errno/errname.go | 2 +- errors.toml | 2 +- executor/revoke.go | 6 ++++++ planner/core/optimizer.go | 4 ++-- planner/core/point_get_plan.go | 2 +- privilege/privileges/privileges_test.go | 16 +++++++++++----- session/session_test.go | 2 +- 7 files changed, 23 insertions(+), 11 deletions(-) diff --git a/errno/errname.go b/errno/errname.go index 5afdbbb91c4c0..62662ce5ac934 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1020,7 +1020,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrBuildExecutor: mysql.Message("Failed to build executor", nil), ErrBatchInsertFail: mysql.Message("Batch insert failed, please clean the table and try again.", nil), ErrGetStartTS: mysql.Message("Can not get start ts", nil), - ErrPrivilegeCheckFail: mysql.Message("privilege check fail", nil), // this error message should begin lowercased to be compatible with the test + ErrPrivilegeCheckFail: mysql.Message("privilege check for '%s' fail", nil), // this error message should begin lowercased to be compatible with the test ErrInvalidWildCard: mysql.Message("Wildcard fields without any table name appears in wrong place", nil), ErrMixOfGroupFuncAndFieldsIncompatible: mysql.Message("In aggregated query without GROUP BY, expression #%d of SELECT list contains nonaggregated column '%s'; this is incompatible with sql_mode=only_full_group_by", nil), ErrUnsupportedSecondArgumentType: mysql.Message("JSON_OBJECTAGG: unsupported second argument type %v", nil), diff --git a/errors.toml b/errors.toml index 458af951629d8..0ce61654373fb 100644 --- a/errors.toml +++ b/errors.toml @@ -1133,7 +1133,7 @@ Schema has changed ["planner:8121"] error = ''' -privilege check fail +privilege check for '%s' fail ''' ["planner:8122"] diff --git a/executor/revoke.go b/executor/revoke.go index 1477534962fbe..b090f048c62a7 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -88,8 +88,14 @@ func (e *RevokeExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } + sessVars := e.ctx.GetSessionVars() // Revoke for each user. for _, user := range e.Users { + if user.User.CurrentUser { + user.User.Username = sessVars.User.AuthUsername + user.User.Hostname = sessVars.User.AuthHostname + } + // Check if user exists. exists, err := userExists(e.ctx, user.User.Username, user.User.Hostname) if err != nil { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d79d83331723b..59c228767171a 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -103,13 +103,13 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [ if v.privilege == mysql.ExtendedPriv { if !pm.RequestDynamicVerification(activeRoles, v.dynamicPriv, v.dynamicWithGrant) { if v.err == nil { - return ErrPrivilegeCheckFail + return ErrPrivilegeCheckFail.GenWithStackByArgs(v.dynamicPriv) } return v.err } } else if !pm.RequestVerification(activeRoles, v.db, v.table, v.column, v.privilege) { if v.err == nil { - return ErrPrivilegeCheckFail + return ErrPrivilegeCheckFail.GenWithStackByArgs(v.privilege.String()) } return v.err } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 77a853a4568b6..f7edbc1648819 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -994,7 +994,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, ch var visitInfos []visitInfo for _, checkType := range checkTypes { if pm != nil && !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", checkType) { - return errors.New("privilege check fail") + return ErrPrivilegeCheckFail.GenWithStackByArgs(checkType.String()) } // This visitInfo is only for table lock check, so we do not need column field, // just fill it empty string. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 2efb565b3ed2b..3038aad397076 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -843,6 +843,12 @@ func (s *testPrivilegeSuite) TestRevokePrivileges(c *C) { c.Assert(se.Auth(&auth.UserIdentity{Username: "hasgrant", Hostname: "localhost", AuthUsername: "hasgrant", AuthHostname: "%"}, nil, nil), IsTrue) mustExec(c, se, "REVOKE SELECT ON mysql.* FROM 'withoutgrant'") mustExec(c, se, "REVOKE ALL ON mysql.* FROM withoutgrant") + + // For issue https://github.com/pingcap/tidb/issues/23850 + mustExec(c, se, "CREATE USER u4") + mustExec(c, se, "GRANT ALL ON *.* TO u4 WITH GRANT OPTION") + c.Assert(se.Auth(&auth.UserIdentity{Username: "u4", Hostname: "localhost", AuthUsername: "u4", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, "REVOKE ALL ON *.* FROM CURRENT_USER()") } func (s *testPrivilegeSuite) TestSetGlobal(c *C) { @@ -1006,14 +1012,14 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) { _, err = se.ExecuteInternal(context.Background(), "drop table information_schema.tables") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "update information_schema.tables set table_name = 'tst' where table_name = 'mysql'") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) // Test performance_schema. mustExec(c, se, `select * from performance_schema.events_statements_summary_by_digest`) _, err = se.ExecuteInternal(context.Background(), "drop table performance_schema.events_statements_summary_by_digest") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "update performance_schema.events_statements_summary_by_digest set schema_name = 'tst'") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "delete from performance_schema.events_statements_summary_by_digest") c.Assert(strings.Contains(err.Error(), "DELETE command denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "create table performance_schema.t(a int)") @@ -1025,7 +1031,7 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) { _, err = se.ExecuteInternal(context.Background(), "drop table metrics_schema.tidb_query_duration") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "update metrics_schema.tidb_query_duration set instance = 'tst'") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "delete from metrics_schema.tidb_query_duration") c.Assert(strings.Contains(err.Error(), "DELETE command denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "create table metric_schema.t(a int)") @@ -1041,9 +1047,9 @@ func (s *testPrivilegeSuite) TestAdminCommand(c *C) { c.Assert(se.Auth(&auth.UserIdentity{Username: "test_admin", Hostname: "localhost"}, nil, nil), IsTrue) _, err := se.ExecuteInternal(context.Background(), "ADMIN SHOW DDL JOBS") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "ADMIN CHECK TABLE t") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue) _, err = se.ExecuteInternal(context.Background(), "ADMIN SHOW DDL JOBS") diff --git a/session/session_test.go b/session/session_test.go index 9ed2f9759243b..5fa7779fc65c3 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2892,7 +2892,7 @@ func (s *testSessionSuite2) TestUpdatePrivilege(c *C) { _, err := tk1.Exec("update t2 set id = 666 where id = 1;") c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) // Cover a bug that t1 and t2 both require update privilege. // In fact, the privlege check for t1 should be update, and for t2 should be select. From ec618f1619bdf473a22861f6f87c486fd0baaa6c Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Wed, 12 May 2021 18:05:40 +0800 Subject: [PATCH 02/27] planner: support set tidb_allow_mpp to `2` or `ENFORCE` to enforce use mpp mode. (#24516) --- executor/mpp_gather.go | 2 +- planner/core/exhaust_physical_plans.go | 6 +- planner/core/integration_test.go | 126 +++++++++++++++++++++++++ planner/core/task.go | 8 +- sessionctx/variable/session.go | 19 +++- sessionctx/variable/sysvar.go | 6 +- sessionctx/variable/tidb_vars.go | 4 +- 7 files changed, 157 insertions(+), 14 deletions(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 64236558af94e..7cfeb613c40f6 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -30,7 +30,7 @@ import ( ) func useMPPExecution(ctx sessionctx.Context, tr *plannercore.PhysicalTableReader) bool { - if !ctx.GetSessionVars().AllowMPPExecution { + if !ctx.GetSessionVars().IsMPPAllowed() { return false } _, ok := tr.GetTablePlan().(*plannercore.PhysicalExchangeSender) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index d4f2923b6220a..cd227657a75d9 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1662,7 +1662,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } joins := make([]PhysicalPlan, 0, 8) canPushToTiFlash := p.canPushToCop(kv.TiFlash) - if p.ctx.GetSessionVars().AllowMPPExecution && canPushToTiFlash { + if p.ctx.GetSessionVars().IsMPPAllowed() && canPushToTiFlash { if p.shouldUseMPPBCJ() { mppJoins := p.tryToGetMppHashJoin(prop, true) if (p.preferJoinType & preferBCJoin) > 0 { @@ -1965,7 +1965,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if !lt.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - if lt.ctx.GetSessionVars().AllowMPPExecution { + if lt.ctx.GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) @@ -2355,7 +2355,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy taskTypes = append(taskTypes, property.CopTiFlashLocalReadTaskType) } canPushDownToTiFlash := la.canPushToCop(kv.TiFlash) - canPushDownToMPP := la.ctx.GetSessionVars().AllowMPPExecution && la.checkCanPushDownToMPP() && canPushDownToTiFlash + canPushDownToMPP := la.ctx.GetSessionVars().IsMPPAllowed() && la.checkCanPushDownToMPP() && canPushDownToTiFlash if la.HasDistinct() { // TODO: remove after the cost estimation of distinct pushdown is implemented. if !la.ctx.GetSessionVars().AllowDistinctAggPushDown { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 7f58d2db6d466..98ab7b7898370 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3706,3 +3706,129 @@ func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { res.Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // test value limit of tidb_opt_tiflash_concurrency_factor + err := tk.ExecToErr("set @@tidb_opt_tiflash_concurrency_factor = 0") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_opt_tiflash_concurrency_factor' can't be set to the value of '0'`) + + tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 1") + tk.MustQuery("select @@tidb_opt_tiflash_concurrency_factor").Check(testkit.Rows("1")) + tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 24") + tk.MustQuery("select @@tidb_opt_tiflash_concurrency_factor").Check(testkit.Rows("24")) + + // test set tidb_allow_mpp + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + tk.MustExec("set @@session.tidb_allow_mpp = 2") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + tk.MustExec("set @@session.tidb_allow_mpp = off") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@session.tidb_allow_mpp = oN") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + tk.MustExec("set @@session.tidb_allow_mpp = enForcE") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + tk.MustExec("set @@global.tidb_allow_mpp = faLsE") + tk.MustQuery("select @@global.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@global.tidb_allow_mpp = True") + tk.MustQuery("select @@global.tidb_allow_mpp").Check(testkit.Rows("ON")) + + err = tk.ExecToErr("set @@global.tidb_allow_mpp = enforceWithTypo") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_allow_mpp' can't be set to the value of 'enforceWithTypo'`) + + // test query + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("create index idx on t(a)") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + // ban mpp + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + + // read from tiflash, batch cop. + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "StreamAgg_20 1.00 285050.00 root funcs:count(Column#5)->Column#3", + "└─TableReader_21 1.00 19003.88 root data:StreamAgg_9", + " └─StreamAgg_9 1.00 19006.88 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_19 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_18 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // open mpp + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + + // should use tikv to index read + tk.MustQuery("explain format='verbose' select count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_30 1.00 485.00 root funcs:count(Column#6)->Column#3", + "└─IndexReader_31 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_29 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tikv, indexRead + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_18 1.00 485.00 root funcs:count(Column#5)->Column#3", + "└─IndexReader_19 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#5", + " └─IndexRangeScan_17 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tiflash, mpp with large cost + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "HashAgg_21 1.00 11910.68 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 11877.08 root data:ExchangeSender_22", + " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // enforce mpp + tk.MustExec("set @@session.tidb_allow_mpp = 2") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + // should use mpp + tk.MustQuery("explain format='verbose' select count(*) from t where a=1;").Check(testkit.Rows( + "HashAgg_24 1.00 33.60 root funcs:count(Column#5)->Column#3", + "└─TableReader_26 1.00 0.00 root data:ExchangeSender_25", + " └─ExchangeSender_25 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_23 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // read from tikv, indexRead + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_18 1.00 485.00 root funcs:count(Column#5)->Column#3", + "└─IndexReader_19 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#5", + " └─IndexRangeScan_17 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tiflash, mpp with little cost + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "HashAgg_21 1.00 33.60 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 0.00 root data:ExchangeSender_22", + " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) +} diff --git a/planner/core/task.go b/planner/core/task.go index 205f5eb77b08a..fa6855503dd0e 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2026,11 +2026,15 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() + + p.cost = t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor + if p.ctx.GetSessionVars().IsMPPEnforced() { + p.cost = 0 + } rt := &rootTask{ p: p, - cst: t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor, + cst: p.cost, } - p.cost = rt.cost() return rt } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 618120b5da6e6..d6bb5763e67d8 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -492,11 +492,12 @@ type SessionVars struct { AllowWriteRowID bool // AllowBatchCop means if we should send batch coprocessor to TiFlash. Default value is 1, means to use batch cop in case of aggregation and join. - // If value is set to 2 , which means to force to send batch cop for any query. Value is set to 0 means never use batch cop. + // Value set to 2 means to force to send batch cop for any query. Value set to 0 means never use batch cop. AllowBatchCop int - // AllowMPPExecution will prefer using mpp way to execute a query. - AllowMPPExecution bool + // AllowMPPExecution means if we should use mpp way to execute query. Default value is "ON", means to be determined by the optimizer. + // Value set to "ENFORCE" means to use mpp whenever possible. Value set to means never use mpp. + allowMPPExecution string // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. AllowAutoRandExplicitInsert bool @@ -845,6 +846,16 @@ func (s *SessionVars) AllocMPPTaskID(startTS uint64) int64 { return 1 } +// IsMPPAllowed returns whether mpp execution is allowed. +func (s *SessionVars) IsMPPAllowed() bool { + return s.allowMPPExecution != "OFF" +} + +// IsMPPEnforced returns whether mpp execution is enforced. +func (s *SessionVars) IsMPPEnforced() bool { + return s.allowMPPExecution == "ENFORCE" +} + // CheckAndGetTxnScope will return the transaction scope we should use in the current session. func (s *SessionVars) CheckAndGetTxnScope() string { if s.InRestrictedSQL { @@ -1094,7 +1105,7 @@ func NewSessionVars() *SessionVars { terror.Log(vars.SetSystemVar(TiDBEnableStreaming, enableStreaming)) vars.AllowBatchCop = DefTiDBAllowBatchCop - vars.AllowMPPExecution = DefTiDBAllowMPPExecution + vars.allowMPPExecution = DefTiDBAllowMPPExecution var enableChunkRPC string if config.GetGlobalConfig().TiKVClient.EnableChunkRPC { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9a01c19470722..73a8ca0066450 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -720,8 +720,8 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution), SetSession: func(s *SessionVars, val string) error { - s.AllowMPPExecution = TiDBOptOn(val) + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { + s.allowMPPExecution = val return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdCount, Value: strconv.Itoa(DefBroadcastJoinThresholdCount), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { @@ -793,7 +793,7 @@ var defaultSysVars = []*SysVar{ s.CPUFactor = tidbOptFloat64(val, DefOptCPUFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, DefOptTiFlashConcurrencyFactor) return nil }}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 54616f902db5e..e416f9a695fc3 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -290,6 +290,8 @@ const ( // The default value is 0 TiDBAllowBatchCop = "tidb_allow_batch_cop" + // TiDBAllowMPPExecution means if we should use mpp way to execute query. Default value is 1 (or 'ON'), means to be determined by the optimizer. + // Value set to 2 (or 'ENFORCE') which means to use mpp whenever possible. Value set to 2 (or 'OFF') means never use mpp. TiDBAllowMPPExecution = "tidb_allow_mpp" // TiDBInitChunkSize is used to control the init chunk size during query execution. @@ -613,7 +615,7 @@ const ( DefBroadcastJoinThresholdCount = 10 * 1024 DefTiDBOptimizerSelectivityLevel = 0 DefTiDBAllowBatchCop = 1 - DefTiDBAllowMPPExecution = true + DefTiDBAllowMPPExecution = "ON" DefTiDBTxnMode = "" DefTiDBRowFormatV1 = 1 DefTiDBRowFormatV2 = 2 From 01798548d973bae07109bd2b72a7087678e78f52 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 18:27:40 +0800 Subject: [PATCH 03/27] store/tikv: remove use of SchemaAmender option in store/tikv (#24408) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/txn.go | 9 +++++---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 50bba80d2b54e..24e6f11c8fef9 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -148,6 +148,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) + case tikvstore.SchemaAmender: + txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) case tikvstore.Enable1PC: diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 1d678d010957e..47ccdce12caea 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -223,10 +223,6 @@ func (txn *KVTxn) Delete(k []byte) error { func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) txn.snapshot.SetOption(opt, val) - switch opt { - case tikv.SchemaAmender: - txn.schemaAmender = val.(SchemaAmender) - } } // GetOption returns the option @@ -265,6 +261,11 @@ func (txn *KVTxn) SetPriority(pri Priority) { txn.GetSnapshot().SetPriority(pri) } +// SetSchemaAmender sets an amender to update mutations after schema change. +func (txn *KVTxn) SetSchemaAmender(sa SchemaAmender) { + txn.schemaAmender = sa +} + // SetCommitCallback sets up a function that will be called when the transaction // is finished. func (txn *KVTxn) SetCommitCallback(f func(string, error)) { From 081291b06ef7bc121fb9abbc60648bd7dd5759a9 Mon Sep 17 00:00:00 2001 From: Howie Date: Wed, 12 May 2021 18:51:40 +0800 Subject: [PATCH 04/27] *: the value of tikv-client.store-liveness-timeout should not less than 0 (#24244) --- tidb-server/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tidb-server/main.go b/tidb-server/main.go index f070d2eeec48d..3e2351bf7c352 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -572,7 +572,7 @@ func setGlobalVars() { kvcache.GlobalLRUMemUsageTracker.AttachToGlobalTracker(executor.GlobalMemoryUsageTracker) t, err := time.ParseDuration(cfg.TiKVClient.StoreLivenessTimeout) - if err != nil { + if err != nil || t < 0 { logutil.BgLogger().Fatal("invalid duration value for store-liveness-timeout", zap.String("currentValue", cfg.TiKVClient.StoreLivenessTimeout)) } From 1ae648bd61a64aae18ee63e82c57720983968c61 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 19:11:40 +0800 Subject: [PATCH 05/27] store/tikv: remove use of EnableAsyncCommit option in store/tikv (#24462) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 4 +--- store/tikv/tests/2pc_test.go | 2 +- store/tikv/tests/async_commit_test.go | 4 ++-- store/tikv/tests/snapshot_fail_test.go | 5 +++-- store/tikv/txn.go | 6 ++++++ 6 files changed, 15 insertions(+), 8 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 24e6f11c8fef9..739b983fd3d99 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -152,6 +152,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) + case tikvstore.EnableAsyncCommit: + txn.SetEnableAsyncCommit(val.(bool)) case tikvstore.Enable1PC: txn.SetEnable1PC(val.(bool)) case tikvstore.TxnScope: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 8703b1861c65d..ee94eceec166a 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -825,12 +825,10 @@ func (c *twoPhaseCommitter) checkAsyncCommit() bool { return false } - enableAsyncCommitOption := c.txn.us.GetOption(kv.EnableAsyncCommit) - enableAsyncCommit := enableAsyncCommitOption != nil && enableAsyncCommitOption.(bool) asyncCommitCfg := config.GetGlobalConfig().TiKVClient.AsyncCommit // TODO the keys limit need more tests, this value makes the unit test pass by now. // Async commit is not compatible with Binlog because of the non unique timestamp issue. - if c.sessionID > 0 && enableAsyncCommit && + if c.sessionID > 0 && c.txn.enableAsyncCommit && uint(c.mutations.Len()) <= asyncCommitCfg.KeysLimit && !c.shouldWriteBinlog() { totalKeySize := uint64(0) diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index d1e635f205efa..5589752043b2b 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -105,7 +105,7 @@ func (s *testCommitterSuite) begin(c *C) tikv.TxnProbe { func (s *testCommitterSuite) beginAsyncCommit(c *C) tikv.TxnProbe { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) return txn } diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 0f4985fa7ab86..381771bfa0836 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -134,7 +134,7 @@ func (s *testAsyncCommitCommon) beginAsyncCommitWithLinearizability(c *C) tikv.T func (s *testAsyncCommitCommon) beginAsyncCommit(c *C) tikv.TxnProbe { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) return tikv.TxnProbe{KVTxn: txn} } @@ -160,7 +160,7 @@ func (s *testAsyncCommitSuite) SetUpTest(c *C) { func (s *testAsyncCommitSuite) lockKeysWithAsyncCommit(c *C, keys, values [][]byte, primaryKey, primaryValue []byte, commitPrimary bool) (uint64, uint64) { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) for i, k := range keys { if len(values[i]) > 0 { err = txn.Set(k, values[i]) diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 1360841bd743a..aca3c59099cf7 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -152,6 +152,7 @@ func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/twoPCShortLockTTL", "return"), IsNil) @@ -181,7 +182,7 @@ func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { // Prewrite k1 and k2 again without committing them txn, err = s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) err = txn.Set([]byte("k1"), []byte("v3")) c.Assert(err, IsNil) err = txn.Set([]byte("k2"), []byte("v4")) @@ -210,7 +211,7 @@ func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { c.Assert(txn.Set([]byte("k1"), []byte("v1")), IsNil) err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, false) + txn.SetEnableAsyncCommit(false) txn.SetEnable1PC(false) txn.SetOption(kv.GuaranteeLinearizability, false) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 47ccdce12caea..a8c0f70f8da8d 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -82,6 +82,7 @@ type KVTxn struct { syncLog bool priority Priority isPessimistic bool + enableAsyncCommit bool enable1PC bool scope string kvFilter KVFilter @@ -272,6 +273,11 @@ func (txn *KVTxn) SetCommitCallback(f func(string, error)) { txn.commitCallback = f } +// SetEnableAsyncCommit indicates if the transaction will try to use async commit. +func (txn *KVTxn) SetEnableAsyncCommit(b bool) { + txn.enableAsyncCommit = b +} + // SetEnable1PC indicates if the transaction will try to use 1 phase commit. func (txn *KVTxn) SetEnable1PC(b bool) { txn.enable1PC = b From b1d134d7d9264a72df902087866b6de23d4efb93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Wed, 12 May 2021 19:25:40 +0800 Subject: [PATCH 06/27] txn: Add txn state's view (#22908) --- bindinfo/bind_test.go | 5 ++ domain/domain_test.go | 5 ++ executor/adapter.go | 3 + executor/builder.go | 4 +- executor/executor_pkg_test.go | 5 ++ executor/explainfor_test.go | 5 ++ executor/infoschema_reader.go | 38 +++++++++ executor/infoschema_reader_test.go | 5 ++ executor/prepared_test.go | 5 ++ executor/seqtest/prepared_test.go | 5 ++ infoschema/cluster.go | 3 + infoschema/infoschema_test.go | 1 + infoschema/tables.go | 38 +++++++-- infoschema/tables_test.go | 36 +++++++- kv/kv.go | 1 + server/server.go | 19 ++++- session/session.go | 24 +++++- session/session_test.go | 102 ++++++++++++++++++++++ session/txn.go | 133 +++++++++++++++++++++++------ session/txninfo/txn_info.go | 96 +++++++++++++++++++++ util/processinfo.go | 2 + 21 files changed, 493 insertions(+), 42 deletions(-) create mode 100644 session/txninfo/txn_info.go diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 1eb7e1478b2f9..22b60187f0a77 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/util" @@ -70,6 +71,10 @@ type mockSessionManager struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) for _, item := range msm.PS { diff --git a/domain/domain_test.go b/domain/domain_test.go index 7c9d9ff633bc5..a4432b0fb1fe6 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" @@ -241,6 +242,10 @@ type mockSessionManager struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) for _, item := range msm.PS { diff --git a/executor/adapter.go b/executor/adapter.go index 5f5229195c3f9..44d00cd1efa1e 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" + "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -377,6 +378,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if txn.Valid() { txnStartTS = txn.StartTS() } + return &recordSet{ executor: e, stmt: a, @@ -590,6 +592,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } e, err = a.handlePessimisticLockError(ctx, err) if err != nil { + // todo: Report deadlock if ErrDeadlock.Equal(err) { metrics.StatementDeadlockDetectDuration.Observe(time.Since(startLocking).Seconds()) } diff --git a/executor/builder.go b/executor/builder.go index 40282d1030b2c..3324e52f894ff 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1531,7 +1531,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TablePlacementPolicy), strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), strings.ToLower(infoschema.TableClientErrorsSummaryByUser), - strings.ToLower(infoschema.TableClientErrorsSummaryByHost): + strings.ToLower(infoschema.TableClientErrorsSummaryByHost), + strings.ToLower(infoschema.TableTiDBTrx), + strings.ToLower(infoschema.ClusterTableTiDBTrx): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 7cc5a8a69d66e..5591dcefde54d 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" plannerutil "github.com/pingcap/tidb/planner/util" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -60,6 +61,10 @@ type mockSessionManager struct { serverID uint64 } +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index a113200a925d8..e29a7a3e24cee 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" @@ -38,6 +39,10 @@ type mockSessionManager1 struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo { + return nil +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 0ec0c48885ecf..ae338bdd644d2 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -149,6 +149,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex infoschema.TableClientErrorsSummaryByUser, infoschema.TableClientErrorsSummaryByHost: err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) + case infoschema.TableTiDBTrx: + e.setDataForTiDBTrx(sctx) + case infoschema.ClusterTableTiDBTrx: + err = e.setDataForClusterTiDBTrx(sctx) } if err != nil { return nil, err @@ -2011,6 +2015,40 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context return nil } +func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) { + sm := ctx.GetSessionManager() + if sm == nil { + return + } + + loginUser := ctx.GetSessionVars().User + var hasProcessPriv bool + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { + hasProcessPriv = true + } + } + infoList := sm.ShowTxnList() + for _, info := range infoList { + // If you have the PROCESS privilege, you can see all running transactions. + // Otherwise, you can see only your own transactions. + if !hasProcessPriv && loginUser != nil && info.Username != loginUser.Username { + continue + } + e.rows = append(e.rows, info.ToDatum()) + } +} + +func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) error { + e.setDataForTiDBTrx(ctx) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) + if err != nil { + return err + } + e.rows = rows + return nil +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index c3e125824873d..e19eb9d9b3064 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" @@ -728,6 +729,10 @@ type mockSessionManager struct { serverID uint64 } +func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { return sm.processInfoMap } diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 1f8edf79d942e..e0e2c19ee0f22 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/domain" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" @@ -135,6 +136,10 @@ type mockSessionManager2 struct { killed bool } +func (sm *mockSessionManager2) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (sm *mockSessionManager2) ShowProcessList() map[uint64]*util.ProcessInfo { pl := make(map[uint64]*util.ProcessInfo) if pi, ok := sm.GetProcessInfo(0); ok { diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 916f218db1f9d..bb8f05e5eff54 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" @@ -796,6 +797,10 @@ type mockSessionManager1 struct { Se session.Session } +func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/infoschema/cluster.go b/infoschema/cluster.go index f113e90a0f587..2d196fe5b0023 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -37,6 +37,8 @@ const ( ClusterTableStatementsSummary = "CLUSTER_STATEMENTS_SUMMARY" // ClusterTableStatementsSummaryHistory is the string constant of cluster statement summary history table. ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY" + // ClusterTableTiDBTrx is the string constant of cluster transaction running table. + ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX" ) // memTableToClusterTables means add memory table to cluster table. @@ -45,6 +47,7 @@ var memTableToClusterTables = map[string]string{ TableProcesslist: ClusterTableProcesslist, TableStatementsSummary: ClusterTableStatementsSummary, TableStatementsSummaryHistory: ClusterTableStatementsSummaryHistory, + TableTiDBTrx: ClusterTableTiDBTrx, } func init() { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index c3892e6527962..6aa0c5526f467 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -332,6 +332,7 @@ func (*testSuite) TestInfoTables(c *C) { "TABLESPACES", "COLLATION_CHARACTER_SET_APPLICABILITY", "PROCESSLIST", + "TIDB_TRX", } for _, t := range infoTables { tb, err1 := is.TableByName(util.InformationSchemaName, model.NewCIStr(t)) diff --git a/infoschema/tables.go b/infoschema/tables.go index bfca649e89fdd..2d5112ada05c0 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -31,11 +31,13 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" @@ -161,6 +163,8 @@ const ( TableClientErrorsSummaryByUser = "CLIENT_ERRORS_SUMMARY_BY_USER" // TableClientErrorsSummaryByHost is the string constant of client errors table. TableClientErrorsSummaryByHost = "CLIENT_ERRORS_SUMMARY_BY_HOST" + // TableTiDBTrx is current running transaction status table. + TableTiDBTrx = "TIDB_TRX" ) var tableIDMap = map[string]int64{ @@ -233,22 +237,25 @@ var tableIDMap = map[string]int64{ TableClientErrorsSummaryGlobal: autoid.InformationSchemaDBID + 67, TableClientErrorsSummaryByUser: autoid.InformationSchemaDBID + 68, TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69, + TableTiDBTrx: autoid.InformationSchemaDBID + 70, + ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 71, } type columnInfo struct { - name string - tp byte - size int - decimal int - flag uint - deflt interface{} - comment string + name string + tp byte + size int + decimal int + flag uint + deflt interface{} + comment string + enumElems []string } func buildColumnInfo(col columnInfo) *model.ColumnInfo { mCharset := charset.CharsetBin mCollation := charset.CharsetBin - if col.tp == mysql.TypeVarchar || col.tp == mysql.TypeBlob || col.tp == mysql.TypeLongBlob { + if col.tp == mysql.TypeVarchar || col.tp == mysql.TypeBlob || col.tp == mysql.TypeLongBlob || col.tp == mysql.TypeEnum { mCharset = charset.CharsetUTF8MB4 mCollation = charset.CollationUTF8MB4 } @@ -259,6 +266,7 @@ func buildColumnInfo(col columnInfo) *model.ColumnInfo { Flen: col.size, Decimal: col.decimal, Flag: col.flag, + Elems: col.enumElems, } return &model.ColumnInfo{ Name: model.NewCIStr(col.name), @@ -1332,6 +1340,19 @@ var tableClientErrorsSummaryByHostCols = []columnInfo{ {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26}, } +var tableTiDBTrxCols = []columnInfo{ + {name: "ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, + {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, + {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, + {name: "STATE", tp: mysql.TypeEnum, enumElems: txninfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, + {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, + {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, + {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, + {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, + {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, + {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1701,6 +1722,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableClientErrorsSummaryGlobal: tableClientErrorsSummaryGlobalCols, TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols, TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols, + TableTiDBTrx: tableTiDBTrxCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index f30f25ba6abfa..6cc24300c1be4 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -28,6 +28,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/fn" + "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -42,6 +43,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" @@ -121,7 +123,7 @@ func (s *testClusterTableSuite) setUpRPCService(c *C, addr string) (*grpc.Server lis, err := net.Listen("tcp", addr) c.Assert(err, IsNil) // Fix issue 9836 - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "root", @@ -276,7 +278,7 @@ func (s *testTableSuite) TestInfoschemaFieldValue(c *C) { tk1.MustQuery("select distinct(table_schema) from information_schema.tables").Check(testkit.Rows("INFORMATION_SCHEMA")) // Fix issue 9836 - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "root", @@ -433,6 +435,11 @@ func (s *testTableSuite) TestCurrentTimestampAsDefault(c *C) { type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo + txnInfo []*txninfo.TxnInfo +} + +func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + return sm.txnInfo } func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { @@ -459,7 +466,7 @@ func (s *testTableSuite) TestSomeTables(c *C) { c.Assert(err, IsNil) tk := testkit.NewTestKit(c, s.store) tk.Se = se - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "user-1", @@ -516,7 +523,7 @@ func (s *testTableSuite) TestSomeTables(c *C) { fmt.Sprintf("3 user-3 127.0.0.1:12345 test Init DB 9223372036 %s %s", "in transaction", "check port"), )) - sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "user-1", @@ -1509,3 +1516,24 @@ func (s *testTableSuite) TestInfoschemaClientErrors(c *C) { err = tk.ExecToErr("FLUSH CLIENT_ERRORS_SUMMARY") c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RELOAD privilege(s) for this operation") } + +func (s *testTableSuite) TestTrx(c *C) { + tk := s.newTestKitWithRoot(c) + _, digest := parser.NormalizeDigest("select * from trx for update;") + sm := &mockSessionManager{nil, make([]*txninfo.TxnInfo, 1)} + sm.txnInfo[0] = &txninfo.TxnInfo{ + StartTS: 424768545227014155, + CurrentSQLDigest: digest, + State: txninfo.TxnRunningNormal, + BlockStartTime: nil, + EntriesCount: 1, + EntriesSize: 19, + ConnectionID: 2, + Username: "root", + CurrentDB: "test", + } + tk.Se.SetSessionManager(sm) + tk.MustQuery("select * from information_schema.TIDB_TRX;").Check( + testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest + " Normal 1 19 2 root test"), + ) +} diff --git a/kv/kv.go b/kv/kv.go index a6a23a88df01d..1fad79d641009 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -154,6 +154,7 @@ type Transaction interface { // String implements fmt.Stringer interface. String() string // LockKeys tries to lock the entries with the keys in KV store. + // Will block until all keys are locked successfully or an error occurs. LockKeys(ctx context.Context, lockCtx *LockCtx, keys ...Key) error // SetOption sets an option with a value, when val is nil, uses the default // value of this option. diff --git a/server/server.go b/server/server.go index f7a6021a11221..29f5307895cc2 100644 --- a/server/server.go +++ b/server/server.go @@ -37,7 +37,6 @@ import ( "math/rand" "net" "net/http" - "unsafe" // For pprof _ "net/http/pprof" @@ -46,6 +45,7 @@ import ( "sync" "sync/atomic" "time" + "unsafe" "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/plugin" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util" @@ -557,6 +558,22 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { return rs } +// ShowTxnList shows all txn info for displaying in `TIDB_TRX` +func (s *Server) ShowTxnList() []*txninfo.TxnInfo { + s.rwlock.RLock() + defer s.rwlock.RUnlock() + rs := make([]*txninfo.TxnInfo, 0, len(s.clients)) + for _, client := range s.clients { + if client.ctx.Session != nil { + info := client.ctx.Session.TxnInfo() + if info != nil { + rs = append(rs, info) + } + } + } + return rs +} + // GetProcessInfo implements the SessionManager interface. func (s *Server) GetProcessInfo(id uint64) (*util.ProcessInfo, bool) { s.rwlock.RLock() diff --git a/session/session.go b/session/session.go index 2f842f92e183a..0b4cb309f434b 100644 --- a/session/session.go +++ b/session/session.go @@ -41,6 +41,9 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tipb/go-binlog" + "go.uber.org/zap" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -58,6 +61,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -81,8 +85,6 @@ import ( "github.com/pingcap/tidb/util/sli" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" - "github.com/pingcap/tipb/go-binlog" - "go.uber.org/zap" ) var ( @@ -145,6 +147,8 @@ type Session interface { Auth(user *auth.UserIdentity, auth []byte, salt []byte) bool AuthWithoutVerification(user *auth.UserIdentity) bool ShowProcess() *util.ProcessInfo + // Return the information of the txn current running + TxnInfo() *txninfo.TxnInfo // PrepareTxnCtx is exported for test. PrepareTxnCtx(context.Context) // FieldList returns fields list of a table. @@ -183,7 +187,7 @@ func (h *StmtHistory) Count() int { type session struct { // processInfo is used by ShowProcess(), and should be modified atomically. processInfo atomic.Value - txn TxnState + txn LazyTxn mu struct { sync.RWMutex @@ -442,6 +446,19 @@ func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { return fields, nil } +func (s *session) TxnInfo() *txninfo.TxnInfo { + txnInfo := s.txn.Info() + if txnInfo == nil { + return nil + } + processInfo := s.ShowProcess() + txnInfo.CurrentSQLDigest = processInfo.Digest + txnInfo.ConnectionID = processInfo.ID + txnInfo.Username = processInfo.User + txnInfo.CurrentDB = processInfo.DB + return txnInfo +} + func (s *session) doCommit(ctx context.Context) error { if !s.txn.Valid() { return nil @@ -524,6 +541,7 @@ func (s *session) doCommit(ctx context.Context) error { if err = memBuffer.Delete(iter.Key()); err != nil { return errors.Trace(err) } + s.txn.UpdateEntriesCountAndSize() if err = iter.Next(); err != nil { return errors.Trace(err) } diff --git a/session/session_test.go b/session/session_test.go index 5fa7779fc65c3..3baee4f0ef6f1 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -42,6 +42,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" @@ -83,6 +84,7 @@ var _ = SerialSuites(&testSessionSerialSuite{}) var _ = SerialSuites(&testBackupRestoreSuite{}) var _ = Suite(&testClusteredSuite{}) var _ = SerialSuites(&testClusteredSerialSuite{}) +var _ = SerialSuites(&testTxnStateSuite{}) type testSessionSuiteBase struct { cluster cluster.Cluster @@ -4303,3 +4305,103 @@ func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { // The global temporary table data is discard after the transaction commit. tk.MustQuery("select * from g_tmp").Check(testkit.Rows()) } + +type testTxnStateSuite struct { + testSessionSuiteBase +} + +func (s *testTxnStateSuite) TestBasic(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info, IsNil) + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t for update;") + info = tk.Se.TxnInfo() + _, expectedDigest := parser.NormalizeDigest("select * from t for update;") + c.Assert(info.CurrentSQLDigest, Equals, expectedDigest) + c.Assert(info.State, Equals, txninfo.TxnRunningNormal) + c.Assert(info.BlockStartTime, IsNil) + // len and size will be covered in TestLenAndSize + c.Assert(info.ConnectionID, Equals, tk.Se.GetSessionVars().ConnectionID) + c.Assert(info.Username, Equals, "") + c.Assert(info.CurrentDB, Equals, "test") + tk.MustExec("commit;") + info = tk.Se.TxnInfo() + c.Assert(info, IsNil) +} + +func (s *testTxnStateSuite) TestEntriesCountAndSize(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("begin pessimistic;") + tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info.EntriesCount, Equals, uint64(1)) + c.Assert(info.EntriesSize, Equals, uint64(29)) + tk.MustExec("insert into t(a) values (2);") + info = tk.Se.TxnInfo() + c.Assert(info.EntriesCount, Equals, uint64(2)) + c.Assert(info.EntriesSize, Equals, uint64(58)) + tk.MustExec("commit;") +} + +func (s *testTxnStateSuite) TestBlocked(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1);") + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t where a = 1 for update;") + go func() { + tk2.MustExec("begin pessimistic") + tk2.MustExec("select * from t where a = 1 for update;") + tk2.MustExec("commit;") + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnLockWaiting) + c.Assert(tk2.Se.TxnInfo().BlockStartTime, NotNil) + tk.MustExec("commit;") +} + +func (s *testTxnStateSuite) TestCommitting(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1), (2);") + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t where a = 1 for update;") + ch := make(chan struct{}) + go func() { + tk2.MustExec("begin pessimistic") + c.Assert(tk2.Se.TxnInfo(), NotNil) + tk2.MustExec("select * from t where a = 2 for update;") + failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)") + defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowCommit") + tk2.MustExec("commit;") + ch <- struct{}{} + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnCommitting) + tk.MustExec("commit;") + <-ch +} + +func (s *testTxnStateSuite) TestRollbacking(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1), (2);") + ch := make(chan struct{}) + go func() { + tk.MustExec("begin pessimistic") + tk.MustExec("insert into t(a) values (3);") + failpoint.Enable("github.com/pingcap/tidb/session/mockSlowRollback", "sleep(200)") + defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowRollback") + tk.MustExec("rollback;") + ch <- struct{}{} + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk.Se.TxnInfo().State, Equals, txninfo.TxnRollingBack) + <-ch +} diff --git a/session/txn.go b/session/txn.go index aebed7ed920b2..133cafb976aae 100644 --- a/session/txn.go +++ b/session/txn.go @@ -20,6 +20,8 @@ import ( "runtime/trace" "strings" "sync/atomic" + "time" + "unsafe" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -28,6 +30,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -39,12 +42,12 @@ import ( "go.uber.org/zap" ) -// TxnState wraps kv.Transaction to provide a new kv.Transaction. +// LazyTxn wraps kv.Transaction to provide a new kv.Transaction. // 1. It holds all statement related modification in the buffer before flush to the txn, // so if execute statement meets error, the txn won't be made dirty. // 2. It's a lazy transaction, that means it's a txnFuture before StartTS() is really need. -type TxnState struct { - // States of a TxnState should be one of the followings: +type LazyTxn struct { + // States of a LazyTxn should be one of the followings: // Invalid: kv.Transaction == nil && txnFuture == nil // Pending: kv.Transaction == nil && txnFuture != nil // Valid: kv.Transaction != nil && txnFuture == nil @@ -55,23 +58,40 @@ type TxnState struct { stagingHandle kv.StagingHandle mutations map[int64]*binlog.TableMutation writeSLI sli.TxnWriteThroughputSLI + + // following atomic fields are used for filling TxnInfo + // we need these fields because kv.Transaction provides no thread safety promise + // but we hope getting TxnInfo is a thread safe op + + infoStartTS uint64 + // current executing state + State txninfo.TxnRunningState + // last trying to block start time + blockStartTime unsafe.Pointer // *time.Time, cannot use atomic.Value here because it is possible to be nil + // how many entries are there in the memBuffer, should be equal to self.(kv.Transaction).Len() + EntriesCount uint64 + // how many memory space do the entries in the memBuffer take, should be equal to self.(kv.Transaction).Size() + EntriesSize uint64 } // GetTableInfo returns the cached index name. -func (txn *TxnState) GetTableInfo(id int64) *model.TableInfo { +func (txn *LazyTxn) GetTableInfo(id int64) *model.TableInfo { return txn.Transaction.GetTableInfo(id) } // CacheTableInfo caches the index name. -func (txn *TxnState) CacheTableInfo(id int64, info *model.TableInfo) { +func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { txn.Transaction.CacheTableInfo(id, info) } -func (txn *TxnState) init() { +func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.EntriesCount, 0) + atomic.StoreUint64(&txn.EntriesSize, 0) } -func (txn *TxnState) initStmtBuf() { +func (txn *LazyTxn) initStmtBuf() { if txn.Transaction == nil { return } @@ -81,14 +101,14 @@ func (txn *TxnState) initStmtBuf() { } // countHint is estimated count of mutations. -func (txn *TxnState) countHint() int { +func (txn *LazyTxn) countHint() int { if txn.stagingHandle == kv.InvalidStagingHandle { return 0 } return txn.Transaction.GetMemBuffer().Len() - txn.initCnt } -func (txn *TxnState) flushStmtBuf() { +func (txn *LazyTxn) flushStmtBuf() { if txn.stagingHandle == kv.InvalidStagingHandle { return } @@ -97,17 +117,19 @@ func (txn *TxnState) flushStmtBuf() { txn.initCnt = buf.Len() } -func (txn *TxnState) cleanupStmtBuf() { +func (txn *LazyTxn) cleanupStmtBuf() { if txn.stagingHandle == kv.InvalidStagingHandle { return } buf := txn.Transaction.GetMemBuffer() buf.Cleanup(txn.stagingHandle) txn.initCnt = buf.Len() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) } // Size implements the MemBuffer interface. -func (txn *TxnState) Size() int { +func (txn *LazyTxn) Size() int { if txn.Transaction == nil { return 0 } @@ -115,19 +137,19 @@ func (txn *TxnState) Size() int { } // Valid implements the kv.Transaction interface. -func (txn *TxnState) Valid() bool { +func (txn *LazyTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() } -func (txn *TxnState) pending() bool { +func (txn *LazyTxn) pending() bool { return txn.Transaction == nil && txn.txnFuture != nil } -func (txn *TxnState) validOrPending() bool { +func (txn *LazyTxn) validOrPending() bool { return txn.txnFuture != nil || txn.Valid() } -func (txn *TxnState) String() string { +func (txn *LazyTxn) String() string { if txn.Transaction != nil { return txn.Transaction.String() } @@ -138,7 +160,7 @@ func (txn *TxnState) String() string { } // GoString implements the "%#v" format for fmt.Printf. -func (txn *TxnState) GoString() string { +func (txn *LazyTxn) GoString() string { var s strings.Builder s.WriteString("Txn{") if txn.pending() { @@ -157,18 +179,25 @@ func (txn *TxnState) GoString() string { return s.String() } -func (txn *TxnState) changeInvalidToValid(kvTxn kv.Transaction) { +func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { txn.Transaction = kvTxn + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.infoStartTS, kvTxn.StartTS()) txn.initStmtBuf() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) txn.txnFuture = nil } -func (txn *TxnState) changeInvalidToPending(future *txnFuture) { +func (txn *LazyTxn) changeInvalidToPending(future *txnFuture) { txn.Transaction = nil txn.txnFuture = future + atomic.StoreUint64(&txn.infoStartTS, 0) + atomic.StoreUint64(&txn.EntriesCount, uint64(0)) + atomic.StoreUint64(&txn.EntriesSize, uint64(0)) } -func (txn *TxnState) changePendingToValid(ctx context.Context) error { +func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { if txn.txnFuture == nil { return errors.New("transaction future is not set") } @@ -183,17 +212,24 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.infoStartTS, t.StartTS()) txn.initStmtBuf() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) return nil } -func (txn *TxnState) changeToInvalid() { +func (txn *LazyTxn) changeToInvalid() { if txn.stagingHandle != kv.InvalidStagingHandle { txn.Transaction.GetMemBuffer().Cleanup(txn.stagingHandle) } txn.stagingHandle = kv.InvalidStagingHandle txn.Transaction = nil txn.txnFuture = nil + atomic.StoreUint64(&txn.infoStartTS, 0) + atomic.StoreUint64(&txn.EntriesCount, 0) + atomic.StoreUint64(&txn.EntriesSize, 0) } var hasMockAutoIncIDRetry = int64(0) @@ -223,7 +259,7 @@ func ResetMockAutoRandIDRetryCount(failTimes int64) { } // Commit overrides the Transaction interface. -func (txn *TxnState) Commit(ctx context.Context) error { +func (txn *LazyTxn) Commit(ctx context.Context) error { defer txn.reset() if len(txn.mutations) != 0 || txn.countHint() != 0 { logutil.BgLogger().Error("the code should never run here", @@ -233,6 +269,10 @@ func (txn *TxnState) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } + atomic.StoreInt32(&txn.State, txninfo.TxnCommitting) + + failpoint.Inject("mockSlowCommit", func(_ failpoint.Value) {}) + // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { if val.(bool) { @@ -259,17 +299,34 @@ func (txn *TxnState) Commit(ctx context.Context) error { } // Rollback overrides the Transaction interface. -func (txn *TxnState) Rollback() error { +func (txn *LazyTxn) Rollback() error { defer txn.reset() + atomic.StoreInt32(&txn.State, txninfo.TxnRollingBack) + // mockSlowRollback is used to mock a rollback which takes a long time + failpoint.Inject("mockSlowRollback", func(_ failpoint.Value) {}) return txn.Transaction.Rollback() } -func (txn *TxnState) reset() { +// LockKeys Wrap the inner transaction's `LockKeys` to record the status +func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { + originState := atomic.LoadInt32(&txn.State) + atomic.StoreInt32(&txn.State, txninfo.TxnLockWaiting) + t := time.Now() + atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(&t)) + err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) + atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(nil)) + atomic.StoreInt32(&txn.State, originState) + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + return err +} + +func (txn *LazyTxn) reset() { txn.cleanup() txn.changeToInvalid() } -func (txn *TxnState) cleanup() { +func (txn *LazyTxn) cleanup() { txn.cleanupStmtBuf() txn.initStmtBuf() for key := range txn.mutations { @@ -278,7 +335,7 @@ func (txn *TxnState) cleanup() { } // KeysNeedToLock returns the keys need to be locked. -func (txn *TxnState) KeysNeedToLock() ([]kv.Key, error) { +func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) { if txn.stagingHandle == kv.InvalidStagingHandle { return nil, nil } @@ -316,6 +373,32 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { return !isNonUniqueIndex } +// Info dump the TxnState to Datum for displaying in `TIDB_TRX` +// This function is supposed to be thread safe +func (txn *LazyTxn) Info() *txninfo.TxnInfo { + startTs := atomic.LoadUint64(&txn.infoStartTS) + if startTs == 0 { + return nil + } + return &txninfo.TxnInfo{ + StartTS: startTs, + State: atomic.LoadInt32(&txn.State), + BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), + EntriesCount: atomic.LoadUint64(&txn.EntriesCount), + EntriesSize: atomic.LoadUint64(&txn.EntriesSize), + } +} + +// UpdateEntriesCountAndSize updates the EntriesCount and EntriesSize +// Note this function is not thread safe, because +// txn.Transaction can be changed during this function's execution if running parallel. +func (txn *LazyTxn) UpdateEntriesCountAndSize() { + if txn.Valid() { + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + } +} + func getBinlogMutation(ctx sessionctx.Context, tableID int64) *binlog.TableMutation { bin := binloginfo.GetPrewriteValue(ctx, true) for i := range bin.Mutations { diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go new file mode 100644 index 0000000000000..77a2d8c90cd05 --- /dev/null +++ b/session/txninfo/txn_info.go @@ -0,0 +1,96 @@ +// Copyright 2021 PingCAP, Inc. + +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package txninfo + +import ( + "time" + + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" +) + +// TxnRunningState is the current state of a transaction +type TxnRunningState = int32 + +const ( + // TxnRunningNormal means the transaction is running normally + TxnRunningNormal TxnRunningState = iota + // TxnLockWaiting means the transaction is blocked on a lock + TxnLockWaiting + // TxnCommitting means the transaction is (at least trying to) committing + TxnCommitting + // TxnRollingBack means the transaction is rolling back + TxnRollingBack +) + +// TxnRunningStateStrs is the names of the TxnRunningStates +var TxnRunningStateStrs = []string{ + "Normal", "LockWaiting", "Committing", "RollingBack", +} + +// TxnInfo is information about a running transaction +// This is supposed to be the datasource of `TIDB_TRX` in infoschema +type TxnInfo struct { + StartTS uint64 + // digest of SQL current running + CurrentSQLDigest string + // current executing State + State TxnRunningState + // last trying to block start time + BlockStartTime *time.Time + // How many entries are in MemDB + EntriesCount uint64 + // MemDB used memory + EntriesSize uint64 + + // the following fields will be filled in `session` instead of `LazyTxn` + + // Which session this transaction belongs to + ConnectionID uint64 + // The user who open this session + Username string + // The schema this transaction works on + CurrentDB string +} + +// ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table +func (info *TxnInfo) ToDatum() []types.Datum { + humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) + var blockStartTime interface{} + if info.BlockStartTime == nil { + blockStartTime = nil + } else { + blockStartTime = types.NewTime(types.FromGoTime(*info.BlockStartTime), mysql.TypeTimestamp, 0) + } + e, err := types.ParseEnumValue(TxnRunningStateStrs, uint64(info.State+1)) + if err != nil { + panic("this should never happen") + } + state := types.NewMysqlEnumDatum(e) + datums := types.MakeDatums( + info.StartTS, + types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), + info.CurrentSQLDigest, + ) + datums = append(datums, state) + datums = append(datums, types.MakeDatums( + blockStartTime, + info.EntriesCount, + info.EntriesSize, + info.ConnectionID, + info.Username, + info.CurrentDB)...) + return datums +} diff --git a/util/processinfo.go b/util/processinfo.go index 29716d914c3de..ebbf17094b80d 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/execdetails" @@ -161,6 +162,7 @@ func serverStatus2Str(state uint16) string { // kill statement rely on this interface. type SessionManager interface { ShowProcessList() map[uint64]*ProcessInfo + ShowTxnList() []*txninfo.TxnInfo GetProcessInfo(id uint64) (*ProcessInfo, bool) Kill(connectionID uint64, query bool) KillAllConnections() From e7db5338101b44ba589fcd264c5247d40ca13882 Mon Sep 17 00:00:00 2001 From: mmyj Date: Wed, 12 May 2021 19:49:40 +0800 Subject: [PATCH 07/27] planner: ignore lock for temporary table of PointGet and BatchPointGet (#24540) --- planner/core/integration_test.go | 26 +++++++++++++++ planner/core/optimizer.go | 25 ++++++++++++++ planner/core/point_get_plan.go | 10 ++++-- .../core/testdata/integration_suite_in.json | 9 +++++ .../core/testdata/integration_suite_out.json | 33 +++++++++++++++++++ 5 files changed, 101 insertions(+), 2 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 98ab7b7898370..9e8eaa9204af9 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3832,3 +3832,29 @@ func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) } + +func (s *testIntegrationSuite) TestEliminateLockForTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test;") + tk.MustExec("create global temporary table t1 (a int primary key, b int, c int, index i_b(b)) on commit delete rows;") + defer func() { + tk.MustExec("drop global temporary table if exists t1;") + }() + tk.MustExec("begin;") + tk.MustExec("insert t1 values (8,8,9);") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 59c228767171a..0c1c4a668d3c8 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -186,6 +187,7 @@ func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { plan = InjectExtraProjection(plan) mergeContinuousSelections(plan) plan = eliminateUnionScanAndLock(sctx, plan) + plan = eliminateLockForTemporaryTable(plan) plan = enableParallelApply(sctx, plan) return plan } @@ -322,6 +324,29 @@ func eliminateUnionScanAndLock(sctx sessionctx.Context, p PhysicalPlan) Physical }) } +// eliminateLockForTemporaryTable eliminates lock for the temporary table. +func eliminateLockForTemporaryTable(p PhysicalPlan) PhysicalPlan { + iteratePhysicalPlan(p, func(p PhysicalPlan) bool { + if len(p.Children()) > 1 { + return false + } + switch x := p.(type) { + case *PointGetPlan: + if x.TblInfo.TempTableType != model.TempTableNone { + x.Lock = false + x.LockWaitTime = 0 + } + case *BatchPointGetPlan: + if x.TblInfo.TempTableType != model.TempTableNone { + x.Lock = false + x.LockWaitTime = 0 + } + } + return true + }) + return p +} + func iteratePhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) bool) { if !f(p) { return diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index f7edbc1648819..fbc0bf9333a29 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -462,7 +462,10 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { if tidbutil.IsMemDB(fp.dbName) { return nil } - fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + // ignore lock for temporary table. + if fp.TblInfo.TempTableType == model.TempTableNone { + fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + } p = fp return } @@ -480,7 +483,10 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { p = tableDual.Init(ctx, &property.StatsInfo{}, 0) return } - fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + // ignore lock for temporary table. + if fp.TblInfo.TempTableType == model.TempTableNone { + fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + } p = fp return } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 087b32110e18f..f386f8d7f24e8 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -294,5 +294,14 @@ "select sum(1) from s1", "select count(1) as cnt from s1 union select count(1) as cnt from s2" ] + }, + { + "name": "TestEliminateLockForTemporaryTable", + "cases": [ + "select * from t1 where a = 2 for update", + "select * from t1 where a in (1,2) for update", + "select c + 1 from t1 where a = 2 and c = 2 for update", + "select c + 1 from t1 where a in (1,2) and c = 2 for update" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 7c735fcb5657c..902ff19276cc0 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1564,5 +1564,38 @@ ] } ] + }, + { + "Name": "TestEliminateLockForTemporaryTable", + "Cases": [ + { + "SQL": "select * from t1 where a = 2 for update", + "Plan": [ + "Point_Get 1.00 root table:t1 handle:2" + ] + }, + { + "SQL": "select * from t1 where a in (1,2) for update", + "Plan": [ + "Batch_Point_Get 2.00 root table:t1 handle:[1 2], keep order:false, desc:false" + ] + }, + { + "SQL": "select c + 1 from t1 where a = 2 and c = 2 for update", + "Plan": [ + "Projection 0.00 root plus(test.t1.c, 1)->Column#4", + "└─Selection 0.00 root eq(test.t1.c, 2)", + " └─Point_Get 1.00 root table:t1 handle:2" + ] + }, + { + "SQL": "select c + 1 from t1 where a in (1,2) and c = 2 for update", + "Plan": [ + "Projection 0.00 root plus(test.t1.c, 1)->Column#4", + "└─Selection 0.00 root eq(test.t1.c, 2)", + " └─Batch_Point_Get 2.00 root table:t1 handle:[1 2], keep order:false, desc:false" + ] + } + ] } ] From 9527fa38dbc5f11d43bce462dbdab219ff9749e2 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 20:01:40 +0800 Subject: [PATCH 08/27] store/tikv: remove use of ReplicaRead transaction option in store/tikv (#24409) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 15 +++++++-------- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 6064db02cdeba..ff8f4b79cc918 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -73,6 +73,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetNotFillCache(val.(bool)) case tikvstore.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) + case tikvstore.ReplicaRead: + s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) default: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 739b983fd3d99..10eef3dbfd7d8 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -144,6 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetPessimistic(val.(bool)) case tikvstore.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.ReplicaRead: + txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) case tikvstore.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index a37e1d8343c5e..d77be2c21ff35 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -565,10 +565,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.ReplicaRead: - s.mu.Lock() - s.mu.replicaRead = val.(kv.ReplicaReadType) - s.mu.Unlock() case kv.CollectRuntimeStats: s.mu.Lock() s.mu.stats = val.(*SnapshotRuntimeStats) @@ -589,10 +585,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { // DelOption deletes an option. func (s *KVSnapshot) DelOption(opt int) { switch opt { - case kv.ReplicaRead: - s.mu.Lock() - s.mu.replicaRead = kv.ReplicaReadLeader - s.mu.Unlock() case kv.CollectRuntimeStats: s.mu.Lock() s.mu.stats = nil @@ -611,6 +603,13 @@ func (s *KVSnapshot) SetKeyOnly(b bool) { s.keyOnly = b } +// SetReplicaRead sets up the replica read type. +func (s *KVSnapshot) SetReplicaRead(readType kv.ReplicaReadType) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.replicaRead = readType +} + // SetIsolationLevel sets the isolation level used to scan data from tikv. func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { s.isolationLevel = level From 6a69ee70ca00fc43b07c3fe33f4b216c5d931606 Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 12 May 2021 20:41:40 +0800 Subject: [PATCH 09/27] store/driver: move error to single package (#24549) --- executor/point_get_test.go | 12 +- server/conn.go | 8 +- server/conn_stmt.go | 4 +- session/pessimistic_test.go | 42 +++---- store/copr/batch_coprocessor.go | 6 +- store/copr/coprocessor.go | 12 +- store/copr/mpp.go | 14 +-- store/copr/store.go | 12 +- store/driver/error/error.go | 158 ++++++++++++++++++++++++++ store/driver/tikv_driver.go | 11 +- store/driver/txn/error.go | 141 +---------------------- store/driver/txn/snapshot.go | 5 +- store/driver/txn/txn_driver.go | 11 +- store/driver/txn/unionstore_driver.go | 23 ++-- 14 files changed, 242 insertions(+), 217 deletions(-) create mode 100644 store/driver/error/error.go diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 846b6f1628fe1..f66446a6bef83 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/tablecodec" @@ -536,15 +536,15 @@ func (s *testPointGetSuite) TestSelectCheckVisibility(c *C) { c.Assert(expectErr.Equal(err), IsTrue) } // Test point get. - checkSelectResultError("select * from t where a='1'", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where a='1'", storeerr.ErrGCTooEarly) // Test batch point get. - checkSelectResultError("select * from t where a in ('1','2')", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where a in ('1','2')", storeerr.ErrGCTooEarly) // Test Index look up read. - checkSelectResultError("select * from t where b > 0 ", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where b > 0 ", storeerr.ErrGCTooEarly) // Test Index read. - checkSelectResultError("select b from t where b > 0 ", txndriver.ErrGCTooEarly) + checkSelectResultError("select b from t where b > 0 ", storeerr.ErrGCTooEarly) // Test table read. - checkSelectResultError("select * from t", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t", storeerr.ErrGCTooEarly) } func (s *testPointGetSuite) TestReturnValues(c *C) { diff --git a/server/conn.go b/server/conn.go index 29c87bd0dfd86..78cdd1a46c12d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -76,7 +76,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/arena" @@ -1569,7 +1569,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { retryable, err = cc.handleStmt(ctx, stmt, parserWarns, i == len(stmts)-1) if err != nil { _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && errors.ErrorEqual(err, storeerr.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. warns := append(parserWarns, stmtctx.SQLWarn{Level: stmtctx.WarnLevelError, Err: err}) @@ -1870,10 +1870,10 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool failpoint.Inject("fetchNextErr", func(value failpoint.Value) { switch value.(string) { case "firstNext": - failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, storeerr.ErrTiFlashServerTimeout) case "secondNext": if !firstNext { - failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, storeerr.ErrTiFlashServerTimeout) } } }) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 242b0df80fc83..e9f56306d9800 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,7 +50,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -198,7 +198,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) retryable, err := cc.executePreparedStmtAndWriteResult(ctx, stmt, args, useCursor) _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, storeerr.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. prevErr := err diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 83f0057384aea..72853d86208a9 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -31,7 +31,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" @@ -611,7 +611,7 @@ func (s *testPessimisticSuite) TestWaitLockKill(c *C) { _, err := tk2.Exec("update test_kill set c = c + 1 where id = 1") wg.Done() c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrQueryInterrupted), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrQueryInterrupted), IsTrue) tk.MustExec("rollback") } @@ -733,10 +733,10 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { timeoutErr := <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) timeoutErr = <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) // tk4 lock c1 = 2 tk4.MustExec("begin pessimistic") @@ -749,7 +749,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err := tk2.Exec("delete from tk where c1 = 2") c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) tk4.MustExec("commit") @@ -767,7 +767,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err = tk2.Exec("delete from tk where c1 = 3") // tk2 tries to lock c1 = 3 fail, this delete should be rollback, but previous update should be keeped c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) tk2.MustExec("commit") tk3.MustExec("commit") @@ -841,7 +841,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/PessimisticLockErrWriteConflict"), IsNil) waitErr := <-done c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Check(duration, GreaterEqual, 1000*time.Millisecond) c.Check(duration, LessEqual, 3000*time.Millisecond) tk2.MustExec("rollback") @@ -1131,11 +1131,11 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err := tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 4 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 7 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") @@ -1147,9 +1147,9 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err = tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 6 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") } @@ -1279,10 +1279,10 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("insert into t1 values(2, 2, 2)") c.Assert(err, NotNil) - c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) + c.Assert(storeerr.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Assert(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk2.MustExec("rollback") } @@ -1429,12 +1429,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("select * from tu where z = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(2, 2);") err = tk2.ExecToErr("select * from tu where z = 4 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) // test batch point get lock tk.MustExec("begin pessimistic") @@ -1443,12 +1443,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err = tk2.ExecToErr("select x from tu where z in (3, 7, 9) for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(5, 6);") err = tk2.ExecToErr("select * from tu where z = 11 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("commit") tk2.MustExec("commit") @@ -1996,11 +1996,11 @@ func (s *testPessimisticSuite) TestSelectForUpdateWaitSeconds(c *C) { waitErr2 := <-errCh waitErr3 := <-errCh c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(waitErr2, NotNil) - c.Check(waitErr2.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr2.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(waitErr3, NotNil) - c.Check(waitErr3.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr3.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(time.Since(start).Seconds(), Less, 45.0) tk2.MustExec("commit") tk3.MustExec("rollback") diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index af224c59fc38a..1e6cdc3cf09bb 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -261,7 +261,7 @@ func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopRe return case <-ticker.C: if atomic.LoadUint32(b.vars.Killed) == 1 { - resp = &batchCopResponse{err: txndriver.ErrQueryInterrupted} + resp = &batchCopResponse{err: derr.ErrQueryInterrupted} ok = true return } @@ -387,7 +387,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } else { logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } - return txndriver.ErrTiFlashServerTimeout + return derr.ErrTiFlashServerTimeout } } } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 5e7eab303e84f..2c1e2d361af76 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -476,7 +476,7 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes return case <-ticker.C: if atomic.LoadUint32(it.vars.Killed) == 1 { - resp = &copResponse{err: txndriver.ErrQueryInterrupted} + resp = &copResponse{err: derr.ErrQueryInterrupted} ok = true return } @@ -717,7 +717,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *backoffer, task *copTask, ch ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) - err = txndriver.ToTiDBErr(err) + err = derr.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { err = worker.handleTiDBSendReqErr(err, task, ch) @@ -874,7 +874,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.R logutil.BgLogger().Debug("coprocessor encounters", zap.Stringer("lock", lockErr)) msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo.TiKVBackoffer(), worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) - err1 = txndriver.ToTiDBErr(err1) + err1 = derr.ToTiDBErr(err1) if err1 != nil { return nil, errors.Trace(err1) } @@ -982,11 +982,11 @@ type CopRuntimeStats struct { func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, ch chan<- *copResponse) error { errCode := errno.ErrUnknown errMsg := err.Error() - if terror.ErrorEqual(err, txndriver.ErrTiKVServerTimeout) { + if terror.ErrorEqual(err, derr.ErrTiKVServerTimeout) { errCode = errno.ErrTiKVServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } - if terror.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) { + if terror.ErrorEqual(err, derr.ErrTiFlashServerTimeout) { errCode = errno.ErrTiFlashServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 10784912faa9b..377e439a9392c 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -225,7 +225,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req if sender.GetRPCError() != nil { logutil.BgLogger().Error("mpp dispatch meet io error", zap.String("error", sender.GetRPCError().Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } } else { @@ -235,7 +235,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req if err != nil { logutil.BgLogger().Error("mpp dispatch meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } @@ -255,7 +255,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } }) @@ -318,7 +318,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques if err != nil { logutil.BgLogger().Error("establish mpp connection meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } @@ -350,7 +350,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } } - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } } @@ -405,7 +405,7 @@ func (m *mppIterator) nextImpl(ctx context.Context) (resp *mppResponse, ok bool, return case <-ticker.C: if m.vars != nil && m.vars.Killed != nil && atomic.LoadUint32(m.vars.Killed) == 1 { - err = txndriver.ErrQueryInterrupted + err = derr.ErrQueryInterrupted exit = true return } diff --git a/store/copr/store.go b/store/copr/store.go index 2cc10ee7bad38..d3f132f85238f 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -39,7 +39,7 @@ func (s *kvStore) GetRegionCache() *tikv.RegionCache { // CheckVisibility checks if it is safe to read using given ts. func (s *kvStore) CheckVisibility(startTime uint64) error { err := s.store.CheckVisibility(startTime) - return txndriver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // GetTiKVClient gets the client instance. @@ -54,13 +54,13 @@ type tikvClient struct { func (c *tikvClient) Close() error { err := c.c.Close() - return txndriver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // SendRequest sends Request. func (c *tikvClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { res, err := c.c.SendRequest(ctx, addr, req, timeout) - return res, txndriver.ToTiDBErr(err) + return res, derr.ToTiDBErr(err) } // Store wraps tikv.KVStore and provides coprocessor utilities. @@ -147,14 +147,14 @@ func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { // It returns a retryable error if total sleep time exceeds maxSleep. func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { e := b.b.Backoff(typ, err) - return txndriver.ToTiDBErr(e) + return derr.ToTiDBErr(e) } // BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message // and never sleep more than maxSleepMs for each sleep. func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) - return txndriver.ToTiDBErr(e) + return derr.ToTiDBErr(e) } // GetBackoffTimes returns a map contains backoff time count by type. diff --git a/store/driver/error/error.go b/store/driver/error/error.go new file mode 100644 index 0000000000000..17da8f7ef2fa3 --- /dev/null +++ b/store/driver/error/error.go @@ -0,0 +1,158 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package error + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/util/dbterror" +) + +// tikv error instance +var ( + // ErrTokenLimit is the error that token is up to the limit. + ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) + // ErrGCTooEarly is the error that GC life time is shorter than transaction duration + ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) + // ErrPDServerTimeout is the error when pd server is timeout. + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) + // ErrUnknown is the unknow error. + ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) +) + +// Registers error returned from TiKV. +var ( + _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) + _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) + _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) +) + +// ToTiDBErr checks and converts a tikv error to a tidb error. +func ToTiDBErr(err error) error { + originErr := err + if err == nil { + return nil + } + err = errors.Cause(err) + if tikverr.IsErrNotFound(err) { + return kv.ErrNotExist + } + + if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { + return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) + } + + if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { + return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { + return kv.ErrCannotSetNilValue + } + + if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { + return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { + return kv.ErrInvalidTxn + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + return ErrTiKVServerTimeout + } + + if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { + if len(e.Error()) == 0 { + return ErrPDServerTimeout + } + return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + return ErrTiFlashServerTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { + return ErrQueryInterrupted + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { + return ErrTiKVServerBusy + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { + return ErrTiFlashServerBusy + } + + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { + return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { + return ErrTiKVStaleCommand + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { + return ErrTiKVMaxTimestampNotSynced + } + + if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { + return ErrLockAcquireFailAndNoWaitSet + } + + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { + return ErrResolveLockTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { + return ErrLockWaitTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { + return ErrRegionUnavailable + } + + if e, ok := err.(*tikverr.ErrTokenLimit); ok { + return ErrTokenLimit.GenWithStackByArgs(e.StoreID) + } + + if errors.ErrorEqual(err, tikverr.ErrUnknown) { + return ErrUnknown + } + + return errors.Trace(originErr) +} diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index cb14736844e68..398be99520aa6 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/copr" + derr "github.com/pingcap/tidb/store/driver/error" txn_driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/tikv" @@ -261,7 +262,7 @@ func (s *tikvStore) StartGCWorker() error { gcWorker, err := gcworker.NewGCWorker(s, s.pdClient) if err != nil { - return txn_driver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } gcWorker.Start() s.gcWorker = gcWorker @@ -286,7 +287,7 @@ func (s *tikvStore) Close() error { } s.coprStore.Close() err := s.KVStore.Close() - return txn_driver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // GetMemCache return memory manager of the storage @@ -298,7 +299,7 @@ func (s *tikvStore) GetMemCache() kv.MemManager { func (s *tikvStore) Begin() (kv.Transaction, error) { txn, err := s.KVStore.Begin() if err != nil { - return nil, txn_driver.ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -307,7 +308,7 @@ func (s *tikvStore) Begin() (kv.Transaction, error) { func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { txn, err := s.KVStore.BeginWithOption(option) if err != nil { - return nil, txn_driver.ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -321,7 +322,7 @@ func (s *tikvStore) GetSnapshot(ver kv.Version) kv.Snapshot { // CurrentVersion returns current max committed version with the given txnScope (local or global). func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { ver, err := s.KVStore.CurrentTimestamp(txnScope) - return kv.NewVersion(ver), txn_driver.ToTiDBErr(err) + return kv.NewVersion(ver), derr.ToTiDBErr(err) } // ShowStatus returns the specified status of the storage diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 4c8e770c44ff7..39931357567be 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -25,56 +25,16 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/dbterror" "go.uber.org/zap" ) -// tikv error instance -var ( - // ErrTokenLimit is the error that token is up to the limit. - ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) - // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) - // ErrGCTooEarly is the error that GC life time is shorter than transaction duration - ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) - // ErrTiKVStaleCommand is the error that the command is stale in tikv. - ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) - // ErrQueryInterrupted is the error when the query is interrupted. - ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) - // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. - ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) - // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. - ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) - // ErrLockWaitTimeout is the error that wait for the lock is timeout. - ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) - // ErrTiKVServerBusy is the error when tikv server is busy. - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) - // ErrTiFlashServerBusy is the error that tiflash server is busy. - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) - // ErrPDServerTimeout is the error when pd server is timeout. - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) - // ErrRegionUnavailable is the error when region is not available. - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) - // ErrUnknown is the unknow error. - ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) -) - -// Registers error returned from TiKV. -var ( - _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) - _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) - _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) -) - func genKeyExistsError(name string, value string, err error) error { if err != nil { logutil.BgLogger().Info("extractKeyExistsErr meets error", zap.Error(err)) @@ -186,104 +146,7 @@ func extractKeyErr(err error) error { notFoundDetail := prettyLockNotFoundKey(e.Retryable) return kv.ErrTxnRetryable.GenWithStackByArgs(e.Retryable + " " + notFoundDetail) } - return ToTiDBErr(err) -} - -// ToTiDBErr checks and converts a tikv error to a tidb error. -func ToTiDBErr(err error) error { - originErr := err - if err == nil { - return nil - } - err = errors.Cause(err) - if tikverr.IsErrNotFound(err) { - return kv.ErrNotExist - } - - if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { - return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) - } - - if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { - return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) - } - - if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { - return kv.ErrCannotSetNilValue - } - - if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { - return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) - } - - if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { - return kv.ErrInvalidTxn - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { - return ErrTiKVServerTimeout - } - - if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { - if len(e.Error()) == 0 { - return ErrPDServerTimeout - } - return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) - } - - if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { - return ErrTiFlashServerTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { - return ErrQueryInterrupted - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { - return ErrTiKVServerBusy - } - - if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { - return ErrTiFlashServerBusy - } - - if e, ok := err.(*tikverr.ErrGCTooEarly); ok { - return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { - return ErrTiKVStaleCommand - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { - return ErrTiKVMaxTimestampNotSynced - } - - if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { - return ErrLockAcquireFailAndNoWaitSet - } - - if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { - return ErrResolveLockTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { - return ErrLockWaitTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { - return ErrRegionUnavailable - } - - if e, ok := err.(*tikverr.ErrTokenLimit); ok { - return ErrTokenLimit.GenWithStackByArgs(e.StoreID) - } - - if errors.ErrorEqual(err, tikverr.ErrUnknown) { - return ErrUnknown - } - - return errors.Trace(originErr) + return derr.ToTiDBErr(err) } func newWriteConflictError(conflict *kvrpcpb.WriteConflict) error { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index ff8f4b79cc918..fde6e93662fa8 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -18,6 +18,7 @@ import ( "unsafe" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) @@ -48,7 +49,7 @@ func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.Iter(k, upperBound) if err != nil { - return nil, ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } @@ -57,7 +58,7 @@ func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.IterReverse(k) if err != nil { - return nil, ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 10eef3dbfd7d8..10186d45756bb 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -75,7 +76,7 @@ func (txn *tikvTxn) GetSnapshot() kv.Snapshot { // The Iterator must be Closed after use. func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.Iter(k, upperBound) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -84,7 +85,7 @@ func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.IterReverse(k) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. @@ -101,17 +102,17 @@ func (txn *tikvTxn) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]b func (txn *tikvTxn) Delete(k kv.Key) error { err := txn.KVTxn.Delete(k) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (txn *tikvTxn) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := txn.KVTxn.Get(ctx, k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (txn *tikvTxn) Set(k kv.Key, v []byte) error { err := txn.KVTxn.Set(k, v) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 9db2325a0148f..5a2f56bfe4233 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/unionstore" ) @@ -39,17 +40,17 @@ func (m *memBuffer) Delete(k kv.Key) error { func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...tikvstore.FlagsOp) error { err := m.MemDB.DeleteWithFlags(k, ops...) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { data, err := m.MemDB.Get(key) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (m *memBuffer) GetFlags(key kv.Key) (tikvstore.KeyFlags, error) { data, err := m.MemDB.GetFlags(key) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (m *memBuffer) Staging() kv.StagingHandle { @@ -73,12 +74,12 @@ func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, tikvsto func (m *memBuffer) Set(key kv.Key, value []byte) error { err := m.MemDB.Set(key, value) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { err := m.MemDB.SetWithFlags(key, value, ops...) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } // Iter creates an Iterator positioned on the first entry that k <= entry's key. @@ -87,7 +88,7 @@ func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) er // The Iterator must be Closed after use. func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := m.MemDB.Iter(k, upperBound) - return &tikvIterator{Iterator: it}, ToTiDBErr(err) + return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -96,7 +97,7 @@ func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (m *memBuffer) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := m.MemDB.IterReverse(k) - return &tikvIterator{Iterator: it}, ToTiDBErr(err) + return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) } // SnapshotIter returns a Iterator for a snapshot of MemBuffer. @@ -121,7 +122,7 @@ func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := u.KVUnionStore.Get(ctx, k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { @@ -134,7 +135,7 @@ func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -143,7 +144,7 @@ func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) // TODO: Add lower bound limit func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } type tikvGetter struct { @@ -156,7 +157,7 @@ func newKVGetter(getter unionstore.Getter) kv.Getter { func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { data, err := g.Getter.Get(k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } // tikvIterator wraps unionstore.Iterator as kv.Iterator From 89c0e6c91a780b894648e042147aafa0b0c1a5e5 Mon Sep 17 00:00:00 2001 From: Howie Date: Wed, 12 May 2021 21:49:40 +0800 Subject: [PATCH 10/27] ddl: add check table compatibility for temporary table (#24501) --- errors.toml | 5 +++++ executor/admin_test.go | 16 ++++++++++++++++ executor/ddl.go | 8 ++++++-- expression/integration_test.go | 2 ++ infoschema/error.go | 2 ++ planner/core/preprocess.go | 27 +++++++++++++++++++++++++++ 6 files changed, 58 insertions(+), 2 deletions(-) diff --git a/errors.toml b/errors.toml index 0ce61654373fb..a54913fa1bd2c 100644 --- a/errors.toml +++ b/errors.toml @@ -1281,6 +1281,11 @@ error = ''' Unknown SEQUENCE: '%-.300s' ''' +["schema:8003"] +error = ''' +TiDB admin check table failed. +''' + ["schema:8020"] error = ''' Table '%s' was locked in %s by %v diff --git a/executor/admin_test.go b/executor/admin_test.go index c9cda897a4745..20095eb59a0ba 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -76,6 +76,22 @@ func (s *testSuite5) TestAdminCheckIndex(c *C) { check() } +func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists temporary_admin_test;") + tk.MustExec("create global temporary table temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2)) ON COMMIT DELETE ROWS;") + tk.MustExec("insert temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") + tk.MustGetErrCode("admin check table temporary_admin_test;", mysql.ErrAdminCheckTable) + tk.MustExec("drop table if exists temporary_admin_test;") + + tk.MustExec("drop table if exists non_temporary_admin_test;") + tk.MustExec("create table non_temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2));") + tk.MustExec("insert non_temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") + tk.MustExec("admin check table non_temporary_admin_test;") + tk.MustExec("drop table if exists non_temporary_admin_test;") +} + func (s *testSuite5) TestAdminRecoverIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/ddl.go b/executor/ddl.go index 81f7221d1e60e..2f10555d21e1e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -311,8 +311,12 @@ func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifEx if isSystemTable(tn.Schema.L, tn.Name.L) { return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L) } - - if obt == tableObject && config.CheckTableBeforeDrop { + tableInfo, err := e.is.TableByName(tn.Schema, tn.Name) + if err != nil { + return err + } + tempTableType := tableInfo.Meta().TempTableType + if obt == tableObject && config.CheckTableBeforeDrop && tempTableType == model.TempTableNone { logutil.BgLogger().Warn("admin check table before drop", zap.String("database", fullti.Schema.O), zap.String("table", fullti.Name.O), diff --git a/expression/integration_test.go b/expression/integration_test.go index f15dc5822be15..a3d983069cce9 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9161,8 +9161,10 @@ func (s *testIntegrationSuite) TestIssue24429(c *C) { tk.MustExec("set @@sql_mode = ANSI_QUOTES;") tk.MustExec("use test") + tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int);") tk.MustQuery(`select t."a"=10 from t;`).Check(testkit.Rows()) + tk.MustExec("drop table if exists t;") } func (s *testIntegrationSuite) TestVitessHash(c *C) { diff --git a/infoschema/error.go b/infoschema/error.go index a0ef7ab9c8760..cb49e48419dec 100644 --- a/infoschema/error.go +++ b/infoschema/error.go @@ -69,4 +69,6 @@ var ( ErrTableLocked = dbterror.ClassSchema.NewStd(mysql.ErrTableLocked) // ErrWrongObject returns when the table/view/sequence is not the expected object. ErrWrongObject = dbterror.ClassSchema.NewStd(mysql.ErrWrongObject) + // ErrAdminCheckTable returns when the check table in temporary mode. + ErrAdminCheckTable = dbterror.ClassSchema.NewStd(mysql.ErrAdminCheckTable) ) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index a3719fe4c4b0b..b5caf55e8de03 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -125,6 +125,8 @@ type preprocessor struct { func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { switch node := in.(type) { + case *ast.AdminStmt: + p.checkAdminCheckTableGrammar(node) case *ast.DeleteStmt: p.stmtTp = TypeDelete case *ast.SelectStmt: @@ -557,6 +559,31 @@ func (p *preprocessor) checkDropDatabaseGrammar(stmt *ast.DropDatabaseStmt) { } } +func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) { + for _, table := range stmt.Tables { + currentDB := p.ctx.GetSessionVars().CurrentDB + if table.Schema.String() != "" { + currentDB = table.Schema.L + } + if currentDB == "" { + p.err = errors.Trace(ErrNoDB) + return + } + sName := model.NewCIStr(currentDB) + tName := table.Name + tableInfo, err := p.is.TableByName(sName, tName) + if err != nil { + p.err = err + return + } + tempTableType := tableInfo.Meta().TempTableType + if stmt.Tp == ast.AdminCheckTable && tempTableType != model.TempTableNone { + p.err = infoschema.ErrAdminCheckTable + return + } + } +} + func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { tName := stmt.Table.Name.String() if isIncorrectName(tName) { From 6228e22467d3233f14e490a890c427db1759d683 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 22:13:40 +0800 Subject: [PATCH 11/27] store/tikv: remove use of IsStatenessReadOnly option in store/tikv (#24464) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 11 +++++++---- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index fde6e93662fa8..95029978ed11e 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -78,6 +78,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) + case tikvstore.IsStalenessReadOnly: + s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 10186d45756bb..200f9e4e18a58 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -161,6 +161,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetEnable1PC(val.(bool)) case tikvstore.TxnScope: txn.SetScope(val.(string)) + case tikvstore.IsStalenessReadOnly: + txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index d77be2c21ff35..4e41aa609aad8 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -571,10 +571,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Unlock() case kv.SampleStep: s.sampleStep = val.(uint32) - case kv.IsStalenessReadOnly: - s.mu.Lock() - s.mu.isStaleness = val.(bool) - s.mu.Unlock() case kv.MatchStoreLabels: s.mu.Lock() s.mu.matchStoreLabels = val.([]*metapb.StoreLabel) @@ -628,6 +624,13 @@ func (s *KVSnapshot) SetTaskID(id uint64) { s.mu.taskID = id } +// SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction +func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.isStaleness = b +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From b6b352cc57c11bea1daae414efbd110db3d0fe64 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 May 2021 22:39:40 +0800 Subject: [PATCH 12/27] store/tikv: change backoff type for missed tiflash peer. (#24577) --- store/copr/batch_coprocessor.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 1e6cdc3cf09bb..4bec370a9a4d5 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -126,8 +126,10 @@ func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.Key if err != nil { return nil, errors.Trace(err) } - // If the region is not found in cache, it must be out - // of date and already be cleaned up. We should retry and generate new tasks. + // When rpcCtx is nil, it's not only attributed to the miss region, but also + // some TiFlash stores crash and can't be recovered. + // That is not an error that can be easily recovered, so we regard this error + // same as rpc error. if rpcCtx == nil { needRetry = true logutil.BgLogger().Info("retry for TiFlash peer with region missing", zap.Uint64("region id", task.region.GetID())) @@ -147,8 +149,10 @@ func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.Key } } if needRetry { - // Backoff once for each retry. - err = bo.Backoff(tikv.BoRegionMiss, errors.New("Cannot find region with TiFlash peer")) + // As mentioned above, nil rpcCtx is always attributed to failed stores. + // It's equal to long poll the store but get no response. Here we'd better use + // TiFlash error to trigger the TiKV fallback mechanism. + err = bo.Backoff(tikv.BoTiFlashRPC, errors.New("Cannot find region with TiFlash peer")) if err != nil { return nil, errors.Trace(err) } From e5840626e5d5ce115030630917916c2901789781 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 23:07:40 +0800 Subject: [PATCH 13/27] store/tikv: remove use of MatchStoreLabels transaction option in store/tikv (#24465) --- store/driver/txn/snapshot.go | 3 +++ store/driver/txn/txn_driver.go | 3 +++ store/tikv/snapshot.go | 11 +++++++---- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 95029978ed11e..6dc2e5c18465c 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -17,6 +17,7 @@ import ( "context" "unsafe" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" @@ -80,6 +81,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetTaskID(val.(uint64)) case tikvstore.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) + case tikvstore.MatchStoreLabels: + s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 200f9e4e18a58..d12a70dc2e2b5 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -19,6 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" @@ -163,6 +164,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetScope(val.(string)) case tikvstore.IsStalenessReadOnly: txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) + case tikvstore.MatchStoreLabels: + txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 4e41aa609aad8..d28ee7ed1fd6b 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -571,10 +571,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Unlock() case kv.SampleStep: s.sampleStep = val.(uint32) - case kv.MatchStoreLabels: - s.mu.Lock() - s.mu.matchStoreLabels = val.([]*metapb.StoreLabel) - s.mu.Unlock() } } @@ -631,6 +627,13 @@ func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { s.mu.isStaleness = b } +// SetMatchStoreLabels sets up labels to filter target stores. +func (s *KVSnapshot) SetMatchStoreLabels(labels []*metapb.StoreLabel) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.matchStoreLabels = labels +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From 771255e0100494e5ff34d8b5c5fcb6c92e0c4ad7 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Wed, 12 May 2021 23:45:40 +0800 Subject: [PATCH 14/27] executor, meta: Allocate auto id for global temporary tables (#24506) --- executor/insert_test.go | 117 +++++++++++++++++++++++++++++++++ sessionctx/variable/session.go | 24 ++++++- table/tables/tables.go | 73 ++++++++++++++++---- util/tableutil/tableutil.go | 40 +++++++++++ 4 files changed, 240 insertions(+), 14 deletions(-) create mode 100644 util/tableutil/tableutil.go diff --git a/executor/insert_test.go b/executor/insert_test.go index bee38e51c0fea..ffcfdc214bdb9 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1590,3 +1590,120 @@ func (s *testSuite10) TestBinaryLiteralInsertToSet(c *C) { tk.MustExec("insert into bintest(h) values(0x61)") tk.MustQuery("select * from bintest").Check(testkit.Rows("a")) } + +var _ = SerialSuites(&testSuite13{&baseTestSuite{}}) + +type testSuite13 struct { + *baseTestSuite +} + +func (s *testSuite13) TestGlobalTempTableAutoInc(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists temp_test") + tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") + defer tk.MustExec("drop table if exists temp_test") + + // Data is cleared after transaction auto commits. + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select * from temp_test").Check(testkit.Rows()) + + // Data is not cleared inside a transaction. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select * from temp_test").Check(testkit.Rows("1")) + tk.MustExec("commit") + + // AutoID allocator is cleared. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select * from temp_test").Check(testkit.Rows("1")) + // Test whether auto-inc is incremental + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("1", "2")) + tk.MustExec("commit") + + // multi-value insert + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("1", "2")) + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec("commit") + + // rebase + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(10)") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("10", "11")) + tk.MustExec("insert into temp_test(id) values(20), (30)") + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("10", "11", "20", "30", "31", "32")) + tk.MustExec("commit") +} + +func (s *testSuite13) TestGlobalTempTableRowID(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists temp_test") + tk.MustExec("create global temporary table temp_test(id int) on commit delete rows") + defer tk.MustExec("drop table if exists temp_test") + + // Data is cleared after transaction auto commits. + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test").Check(testkit.Rows()) + + // Data is not cleared inside a transaction. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test").Check(testkit.Rows("1")) + tk.MustExec("commit") + + // AutoID allocator is cleared. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test").Check(testkit.Rows("1")) + // Test whether row id is incremental + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test order by _tidb_rowid").Check(testkit.Rows("1", "2")) + tk.MustExec("commit") + + // multi-value insert + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select _tidb_rowid from temp_test order by _tidb_rowid").Check(testkit.Rows("1", "2")) + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select _tidb_rowid from temp_test order by _tidb_rowid").Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec("commit") +} + +func (s *testSuite13) TestGlobalTempTableParallel(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists temp_test") + tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") + defer tk.MustExec("drop table if exists temp_test") + + threads := 8 + loops := 1 + wg := sync.WaitGroup{} + wg.Add(threads) + + insertFunc := func() { + defer wg.Done() + newTk := testkit.NewTestKitWithInit(c, s.store) + newTk.MustExec("begin") + for i := 0; i < loops; i++ { + newTk.MustExec("insert temp_test value(0)") + newTk.MustExec("insert temp_test value(0), (0)") + } + maxID := strconv.Itoa(loops * 3) + newTk.MustQuery("select max(id) from temp_test").Check(testkit.Rows(maxID)) + newTk.MustExec("commit") + } + + for i := 0; i < threads; i++ { + go insertFunc() + } + wg.Wait() +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d6bb5763e67d8..815f42cc03ef3 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" @@ -49,6 +50,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tableutil" "github.com/pingcap/tidb/util/timeutil" "github.com/twmb/murmur3" atomic2 "go.uber.org/atomic" @@ -174,7 +176,9 @@ type TransactionContext struct { // TableDeltaMap lock to prevent potential data race tdmLock sync.Mutex - GlobalTemporaryTables map[int64]struct{} + // GlobalTemporaryTables is used to store transaction-specific information for global temporary tables. + // It can also be stored in sessionCtx with local temporary tables, but it's easier to clean this data after transaction ends. + GlobalTemporaryTables map[int64]tableutil.TempTable } // GetShard returns the shard prefix for the next `count` rowids. @@ -1456,6 +1460,24 @@ func (s *SessionVars) LazyCheckKeyNotExists() bool { return s.PresumeKeyNotExists || (s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning) } +// GetTemporaryTable returns a TempTable by tableInfo. +func (s *SessionVars) GetTemporaryTable(tblInfo *model.TableInfo) tableutil.TempTable { + if tblInfo.TempTableType == model.TempTableGlobal { + if s.TxnCtx.GlobalTemporaryTables == nil { + s.TxnCtx.GlobalTemporaryTables = make(map[int64]tableutil.TempTable) + } + globalTempTables := s.TxnCtx.GlobalTemporaryTables + globalTempTable, ok := globalTempTables[tblInfo.ID] + if !ok { + globalTempTable = tableutil.TempTableFromMeta(tblInfo) + globalTempTables[tblInfo.ID] = globalTempTable + } + return globalTempTable + } + // TODO: check local temporary tables + return nil +} + // special session variables. const ( SQLModeVar = "sql_mode" diff --git a/table/tables/tables.go b/table/tables/tables.go index 8fd3cca9e2657..74fd2d82f3ef9 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" @@ -46,6 +47,7 @@ import ( "github.com/pingcap/tidb/util/generatedexpr" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tableutil" "github.com/pingcap/tipb/go-binlog" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -322,8 +324,8 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, sh := memBuffer.Staging() defer memBuffer.Cleanup(sh) - if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { - addTemporaryTableID(sctx, meta.ID) + if m := t.Meta(); m.TempTableType == model.TempTableGlobal { + addTemporaryTable(sctx, m) } var colIDs, binlogColIDs []int64 @@ -588,12 +590,9 @@ func TryGetCommonPkColumns(tbl table.Table) []*table.Column { return pkCols } -func addTemporaryTableID(sctx sessionctx.Context, id int64) { - txnCtx := sctx.GetSessionVars().TxnCtx - if txnCtx.GlobalTemporaryTables == nil { - txnCtx.GlobalTemporaryTables = make(map[int64]struct{}) - } - txnCtx.GlobalTemporaryTables[id] = struct{}{} +func addTemporaryTable(sctx sessionctx.Context, tblInfo *model.TableInfo) { + tempTable := sctx.GetSessionVars().GetTemporaryTable(tblInfo) + tempTable.SetModified(true) } // AddRecord implements table.Table AddRecord interface. @@ -608,8 +607,8 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . fn.ApplyOn(&opt) } - if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { - addTemporaryTableID(sctx, meta.ID) + if m := t.Meta(); m.TempTableType == model.TempTableGlobal { + addTemporaryTable(sctx, m) } var ctx context.Context @@ -1010,8 +1009,8 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type return err } - if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { - addTemporaryTableID(ctx, meta.ID) + if m := t.Meta(); m.TempTableType == model.TempTableGlobal { + addTemporaryTable(ctx, m) } // The table has non-public column and this column is doing the operation of "modify/change column". @@ -1370,7 +1369,14 @@ func OverflowShardBits(recordID int64, shardRowIDBits uint64, typeBitsLength uin // Allocators implements table.Table Allocators interface. func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { - if ctx == nil || ctx.GetSessionVars().IDAllocator == nil { + if ctx == nil { + return t.allocs + } else if ctx.GetSessionVars().IDAllocator == nil { + // Use an independent allocator for global temporary tables. + if t.meta.TempTableType == model.TempTableGlobal { + alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator() + return autoid.Allocators{alloc} + } return t.allocs } @@ -1498,6 +1504,7 @@ func getDuplicateErrorHandleString(t table.Table, handle kv.Handle, row []types. func init() { table.TableFromMeta = TableFromMeta table.MockTableFromMeta = MockTableFromMeta + tableutil.TempTableFromMeta = TempTableFromMeta } // sequenceCommon cache the sequence value. @@ -1763,3 +1770,43 @@ func BuildTableScanFromInfos(tableInfo *model.TableInfo, columnInfos []*model.Co } return tsExec } + +// TemporaryTable is used to store transaction-specific or session-specific information for global / local temporary tables. +// For example, stats and autoID should have their own copies of data, instead of being shared by all sessions. +type TemporaryTable struct { + // Whether it's modified in this transaction. + modified bool + // The stats of this table. So far it's always pseudo stats. + stats *statistics.Table + // The autoID allocator of this table. + autoIDAllocator autoid.Allocator +} + +// TempTableFromMeta builds a TempTable from model.TableInfo. +func TempTableFromMeta(tblInfo *model.TableInfo) tableutil.TempTable { + return &TemporaryTable{ + modified: false, + stats: statistics.PseudoTable(tblInfo), + autoIDAllocator: autoid.NewAllocatorFromTempTblInfo(tblInfo), + } +} + +// GetAutoIDAllocator is implemented from TempTable.GetAutoIDAllocator. +func (t *TemporaryTable) GetAutoIDAllocator() autoid.Allocator { + return t.autoIDAllocator +} + +// SetModified is implemented from TempTable.SetModified. +func (t *TemporaryTable) SetModified(modified bool) { + t.modified = modified +} + +// GetModified is implemented from TempTable.GetModified. +func (t *TemporaryTable) GetModified() bool { + return t.modified +} + +// GetStats is implemented from TempTable.GetStats. +func (t *TemporaryTable) GetStats() interface{} { + return t.stats +} diff --git a/util/tableutil/tableutil.go b/util/tableutil/tableutil.go new file mode 100644 index 0000000000000..11cbe626dcc56 --- /dev/null +++ b/util/tableutil/tableutil.go @@ -0,0 +1,40 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tableutil + +import ( + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/meta/autoid" +) + +// TempTable is used to store transaction-specific or session-specific information for global / local temporary tables. +// For example, stats and autoID should have their own copies of data, instead of being shared by all sessions. +type TempTable interface { + // GetAutoIDAllocator gets the autoID allocator of this table. + GetAutoIDAllocator() autoid.Allocator + + // SetModified sets that the table is modified. + SetModified(bool) + + // GetModified queries whether the table is modified. + GetModified() bool + + // The stats of this table (*statistics.Table). + // Define the return type as interface{} here to avoid cycle imports. + GetStats() interface{} +} + +// TempTableFromMeta builds a TempTable from *model.TableInfo. +// Currently, it is assigned to tables.TempTableFromMeta in tidb package's init function. +var TempTableFromMeta func(tblInfo *model.TableInfo) TempTable From d54e8841a130a6c1d2b0f5ad13d13d4590f1d55b Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 00:27:40 +0800 Subject: [PATCH 15/27] store/tikv: remove use of SampleStep option in store/tikv (#24461) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 7 +++++-- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 6dc2e5c18465c..ee1d1eeee29d8 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -77,6 +77,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetSnapshotTS(val.(uint64)) case tikvstore.ReplicaRead: s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) + case tikvstore.SampleStep: + s.KVSnapshot.SetSampleStep(val.(uint32)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) case tikvstore.IsStalenessReadOnly: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index d12a70dc2e2b5..12f2c8233ccb1 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -154,6 +154,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) + case tikvstore.SampleStep: + txn.KVTxn.GetSnapshot().SetSampleStep(val.(uint32)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) case tikvstore.EnableAsyncCommit: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index d28ee7ed1fd6b..ae65f15dc18e6 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -569,8 +569,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Lock() s.mu.stats = val.(*SnapshotRuntimeStats) s.mu.Unlock() - case kv.SampleStep: - s.sampleStep = val.(uint32) } } @@ -607,6 +605,11 @@ func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { s.isolationLevel = level } +// SetSampleStep skips 'step - 1' number of keys after each returned key. +func (s *KVSnapshot) SetSampleStep(step uint32) { + s.sampleStep = step +} + // SetPriority sets the priority for tikv to execute commands. func (s *KVSnapshot) SetPriority(pri Priority) { s.priority = pri From 61131c6f1df18b1e696025f9f034940df0596142 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 13 May 2021 00:47:39 +0800 Subject: [PATCH 16/27] executor: add partition pruning tests for adding and dropping partition operations (#24573) --- executor/partition_table_test.go | 43 ++++++++++++++++++++++++++++---- util/testkit/testkit.go | 8 +++--- 2 files changed, 43 insertions(+), 8 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 0c9e63129ebf1..5be39c3a04d54 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -494,12 +494,45 @@ func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { for _, t := range tests { for i := range t.partitions { sql := fmt.Sprintf(t.sql, tables[i]) - c.Assert(tk.MustPartition(sql, t.partitions[i]), IsTrue) - tk.MustQuery(sql).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) + tk.MustPartition(sql, t.partitions[i]).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) } } } +func (s *partitionTableSuite) TestAddDropPartitions(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_add_drop_partition") + tk.MustExec("use test_add_drop_partition") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int) partition by range(a) ( + partition p0 values less than (5), + partition p1 values less than (10), + partition p2 values less than (15))`) + tk.MustExec(`insert into t values (2), (7), (12)`) + tk.MustPartition(`select * from t where a < 3`, "p0").Sort().Check(testkit.Rows("2")) + tk.MustPartition(`select * from t where a < 8`, "p0,p1").Sort().Check(testkit.Rows("2", "7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "2", "7")) + + // remove p0 + tk.MustExec(`alter table t drop partition p0`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "7")) + + // add 2 more partitions + tk.MustExec(`alter table t add partition (partition p3 values less than (20))`) + tk.MustExec(`alter table t add partition (partition p4 values less than (40))`) + tk.MustExec(`insert into t values (15), (25)`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") @@ -511,15 +544,15 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") // list partition table - tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( + tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( partition p0 values in (1, 2, 3, 4), partition p1 values in (5, 6, 7, 8), partition p2 values in (9, 10, 11, 12));`) // range partition table tk.MustExec(`create table trange(a int, b int, index idx_a(a), index idx_b(b)) partition by range(a) ( - partition p0 values less than(300), - partition p1 values less than (500), + partition p0 values less than(300), + partition p1 values less than (500), partition p2 values less than(1100));`) // hash partition table diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 06eb826c56b78..d6d6e41bb6e9f 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -256,14 +256,16 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { } // MustPartition checks if the result execution plan must read specific partitions. -func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) bool { +func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) + ok := false for i := range rs.rows { if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { - return true + ok = true } } - return false + tk.c.Assert(ok, check.IsTrue) + return tk.MustQuery(sql, args...) } // MustUseIndex checks if the result execution plan contains specific index(es). From 75fcd6586ba6e26da1a63f62d9aff007b7a25284 Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 13 May 2021 02:19:39 +0800 Subject: [PATCH 17/27] ddl: forbid partition on temporary mode before put into queue (#24565) --- ddl/ddl_api.go | 5 ++++- ddl/partition.go | 7 +++++++ ddl/table.go | 5 ----- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b03b4ca66f536..e6d77c9e674e9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1617,7 +1617,10 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T return errors.Trace(err) } if err = checkAddPartitionTooManyPartitions(uint64(len(tbInfo.Partition.Definitions))); err != nil { - return errors.Trace(err) + return err + } + if err = checkAddPartitionOnTemporaryMode(tbInfo); err != nil { + return err } switch tbInfo.Partition.Type { diff --git a/ddl/partition.go b/ddl/partition.go index 4cc71eb1c8d74..0cafa9d2ff525 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1476,6 +1476,13 @@ func checkAddPartitionTooManyPartitions(piDefs uint64) error { return nil } +func checkAddPartitionOnTemporaryMode(tbInfo *model.TableInfo) error { + if tbInfo.Partition != nil && tbInfo.TempTableType != model.TempTableNone { + return ErrPartitionNoTemporary + } + return nil +} + func checkNoHashPartitions(ctx sessionctx.Context, partitionNum uint64) error { if partitionNum == 0 { return ast.ErrNoParts.GenWithStackByArgs("partitions") diff --git a/ddl/table.go b/ddl/table.go index acd209a7bb0da..668de3ac41c05 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -56,11 +56,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) job.State = model.JobStateCancelled return ver, errors.Trace(err) } - if tbInfo.Partition != nil && (tbInfo.TempTableType == model.TempTableGlobal || tbInfo.TempTableType == model.TempTableLocal) { - // unsupported ddl, cancel this job. - job.State = model.JobStateCancelled - return ver, errors.Trace(ErrPartitionNoTemporary) - } tbInfo.State = model.StateNone err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) From f92df3c5b17cae2784c14863dca996c841ef5cd7 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Thu, 13 May 2021 10:35:40 +0800 Subject: [PATCH 18/27] ddl: speedup test case TestIndexOnMultipleGeneratedColumn (#24487) --- ddl/db_integration_test.go | 42 ++++++++++++++++++++++++++++++++------ 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index fe6bca7dc4563..405ada57f15ec 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -969,9 +969,9 @@ func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) { func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test") tk.MustExec("use test") + tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1))") tk.MustExec("insert into t (a) values (1)") @@ -980,42 +980,72 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { res := tk.MustQuery("select * from t use index(idx) where c > 1") tk.MustQuery("select * from t ignore index(idx) where c > 1").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn1(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1), d int as (c + 1))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn2(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bigint, b decimal as (a+1), c varchar(20) as (b*2), d float as (a*23+b-1+length(c)))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 4 25")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn3(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(10), b float as (length(a)+123), c varchar(20) as (right(a, 2)), d float as (b+b-7+1-3+3*ASCII(c)))") tk.MustExec("insert into t (a) values ('adorable')") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("adorable 131 le 577")) // 131+131-7+1-3+3*108 - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn4(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bigint, b decimal as (a), c int(10) as (a+b), d float as (a+b+c), e decimal as (a+b+c+d))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 1 2 4 8")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a bigint, b bigint as (a+1) virtual, c bigint as (b+1) virtual)") @@ -1025,7 +1055,7 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk.MustExec("alter table t add column(d bigint as (c+1) virtual)") tk.MustExec("alter table t add index idx_d(d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4")) - res = tk.MustQuery("select * from t use index(idx_d) where d > 2") + res := tk.MustQuery("select * from t use index(idx_d) where d > 2") tk.MustQuery("select * from t ignore index(idx_d) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") } From dc5c2264234f6135cbdae32a6f8c6ea54331c2ba Mon Sep 17 00:00:00 2001 From: sylzd Date: Thu, 13 May 2021 11:15:39 +0800 Subject: [PATCH 19/27] execution: Fix issue 24439 Inconsistent error with MySQL for GRANT CREATE USER ON .* (#24485) --- errors.toml | 5 +++++ executor/errors.go | 1 + executor/grant.go | 6 ++++++ executor/grant_test.go | 4 ++++ go.mod | 2 +- go.sum | 4 ++-- 6 files changed, 19 insertions(+), 3 deletions(-) diff --git a/errors.toml b/errors.toml index a54913fa1bd2c..926823909f96e 100644 --- a/errors.toml +++ b/errors.toml @@ -501,6 +501,11 @@ error = ''' Deadlock found when trying to get lock; try restarting transaction ''' +["executor:1221"] +error = ''' +Incorrect usage of %s and %s +''' + ["executor:1242"] error = ''' Subquery returns more than 1 row diff --git a/executor/errors.go b/executor/errors.go index 7f3345659e4f9..ad8104a96e7ee 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -42,6 +42,7 @@ var ( ErrTableaccessDenied = dbterror.ClassExecutor.NewStd(mysql.ErrTableaccessDenied) ErrBadDB = dbterror.ClassExecutor.NewStd(mysql.ErrBadDB) ErrWrongObject = dbterror.ClassExecutor.NewStd(mysql.ErrWrongObject) + ErrWrongUsage = dbterror.ClassExecutor.NewStd(mysql.ErrWrongUsage) ErrRoleNotGranted = dbterror.ClassPrivilege.NewStd(mysql.ErrRoleNotGranted) ErrDeadlock = dbterror.ClassExecutor.NewStd(mysql.ErrLockDeadlock) ErrQueryInterrupted = dbterror.ClassExecutor.NewStd(mysql.ErrQueryInterrupted) diff --git a/executor/grant.go b/executor/grant.go index 6c715758b6c7f..86620cc0124b1 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -473,6 +473,12 @@ func (e *GrantExec) grantDBLevel(priv *ast.PrivElem, user *ast.UserSpec, interna if priv.Priv == mysql.UsagePriv { return nil } + for _, v := range mysql.StaticGlobalOnlyPrivs { + if v == priv.Priv { + return ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES") + } + } + dbName := e.Level.DBName if len(dbName) == 0 { dbName = e.ctx.GetSessionVars().CurrentDB diff --git a/executor/grant_test.go b/executor/grant_test.go index bb720b48b730b..13686494feddc 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -89,6 +89,10 @@ func (s *testSuite3) TestGrantDBScope(c *C) { sql := fmt.Sprintf("SELECT %s FROM mysql.DB WHERE User=\"testDB1\" and host=\"localhost\" and db=\"test\";", mysql.Priv2UserCol[v]) tk.MustQuery(sql).Check(testkit.Rows("Y")) } + + // Grant in wrong scope. + _, err := tk.Exec(` grant create user on test.* to 'testDB1'@'localhost';`) + c.Assert(terror.ErrorEqual(err, executor.ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES")), IsTrue) } func (s *testSuite3) TestWithGrantOption(c *C) { diff --git a/go.mod b/go.mod index cd8becf5a757c..b2bd664da13f3 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c + github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index 4e1030039a04f..c42d129ae5b8b 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c h1:GLFd+wBN7EsV6ad/tVGFCD37taOyzIMVs3SdiWZF18I= -github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b h1:eLuDQ6eJCEKCbGwhGrkjzagwev1GJGU2Y2kFkAsBzV0= +github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= From d5a0e6a1d32c9e1a270f8b168c65db527c9643b4 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Thu, 13 May 2021 11:31:39 +0800 Subject: [PATCH 20/27] *: fix errcheck (#24463) --- store/tikv/config/config_test.go | 13 ++++++------- types/json/binary_functions_test.go | 2 +- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/store/tikv/config/config_test.go b/store/tikv/config/config_test.go index f79f2d09c22c8..a47e1e7e5030b 100644 --- a/store/tikv/config/config_test.go +++ b/store/tikv/config/config_test.go @@ -34,20 +34,19 @@ func (s *testConfigSuite) TestParsePath(c *C) { } func (s *testConfigSuite) TestTxnScopeValue(c *C) { - - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`), IsNil) isGlobal, v := GetTxnScopeFromConfig() c.Assert(isGlobal, IsFalse) c.Assert(v, Equals, "bj") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`), IsNil) isGlobal, v = GetTxnScopeFromConfig() c.Assert(isGlobal, IsTrue) c.Assert(v, Equals, "global") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`), IsNil) isGlobal, v = GetTxnScopeFromConfig() c.Assert(isGlobal, IsFalse) c.Assert(v, Equals, "global") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) } diff --git a/types/json/binary_functions_test.go b/types/json/binary_functions_test.go index 8191638f7b2f6..12a9c8ece8ae2 100644 --- a/types/json/binary_functions_test.go +++ b/types/json/binary_functions_test.go @@ -35,6 +35,6 @@ func (s *testJSONFuncSuite) TestdecodeEscapedUnicode(c *C) { func BenchmarkDecodeEscapedUnicode(b *testing.B) { for i := 0; i < b.N; i++ { in := "597d" - decodeEscapedUnicode([]byte(in)) + _, _, _ = decodeEscapedUnicode([]byte(in)) } } From 68d0a25183f8f13ee0e010b0a07e2d3516232cae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 13 May 2021 11:55:39 +0800 Subject: [PATCH 21/27] test: make TestExtractStartTs stable (#24585) --- store/tikv/extract_start_ts_test.go | 74 +++++++++++++---------------- store/tikv/kv.go | 14 ++++++ 2 files changed, 47 insertions(+), 41 deletions(-) diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go index a108a0f7e41cb..b392ca365cde8 100644 --- a/store/tikv/extract_start_ts_test.go +++ b/store/tikv/extract_start_ts_test.go @@ -14,9 +14,8 @@ package tikv import ( - "context" - . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/unistore" @@ -28,7 +27,7 @@ type extractStartTsSuite struct { store *KVStore } -var _ = Suite(&extractStartTsSuite{}) +var _ = SerialSuites(&extractStartTsSuite{}) func (s *extractStartTsSuite) SetUpTest(c *C) { client, pdClient, cluster, err := unistore.New("") @@ -63,60 +62,53 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { func (s *extractStartTsSuite) TestExtractStartTs(c *C) { i := uint64(100) - cases := []kv.TransactionOption{ + // to prevent time change during test case execution + // we use failpoint to make it "fixed" + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp", "return(200)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp", `return(300)`), IsNil) + + cases := []struct { + expectedTS uint64 + option kv.TransactionOption + }{ // StartTS setted - {TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + {100, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, // PrevSec setted - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}, + {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, // MinStartTS setted, global - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + {101, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MinStartTS setted, local - {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + {102, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MaxPrevSec setted // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted // see `TestMaxPrevSecFallback` - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, // nothing setted - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, - } - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - stalenessTimestamp, _ := s.store.getStalenessTimestamp(bo, oracle.GlobalTxnScope, 100) - expectedTs := []uint64{ - 100, - stalenessTimestamp, - - 101, - 102, - - stalenessTimestamp, - // it's too hard to figure out the value `getTimestampWithRetry` returns - // so we just check whether it is greater than stalenessTimestamp - 0, + {300, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, } - for i, cs := range cases { - expected := expectedTs[i] - result, _ := extractStartTs(s.store, cs) - if expected == 0 { - c.Assert(result, Greater, stalenessTimestamp) - } else { - c.Assert(result, Equals, expected) - } + for _, cs := range cases { + expected := cs.expectedTS + result, _ := extractStartTs(s.store, cs.option) + c.Assert(result, Equals, expected) } + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp"), IsNil) } func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { s.store.setSafeTS(2, 0x8000000000000002) s.store.setSafeTS(3, 0x8000000000000001) - i := uint64(100) - cases := []kv.TransactionOption{ - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, - {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + cases := []struct { + expectedTS uint64 + option kv.TransactionOption + }{ + {0x8000000000000001, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {0x8000000000000002, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, } - expectedTs := []uint64{0x8000000000000001, 0x8000000000000002} - for i, cs := range cases { - expected := expectedTs[i] - result, _ := extractStartTs(s.store, cs) - c.Assert(result, Equals, expected) + for _, cs := range cases { + result, _ := extractStartTs(s.store, cs.option) + c.Assert(result, Equals, cs.expectedTS) } } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index a487b0024e3e9..981a1b7bc5cab 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -235,6 +235,13 @@ func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { } func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, error) { + failpoint.Inject("MockCurrentTimestamp", func(val failpoint.Value) { + if v, ok := val.(int); ok { + failpoint.Return(uint64(v), nil) + } else { + panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") + } + }) if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -264,6 +271,13 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, } func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec uint64) (uint64, error) { + failpoint.Inject("MockStalenessTimestamp", func(val failpoint.Value) { + if v, ok := val.(int); ok { + failpoint.Return(uint64(v), nil) + } else { + panic("MockStalenessTimestamp should be a number, try use this failpoint with \"return(ts)\"") + } + }) for { startTS, err := s.oracle.GetStaleTimestamp(bo.ctx, txnScope, prevSec) if err == nil { From 5fd39d1ed0e4e5baeb7b759c24e140f7038fcff4 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 13 May 2021 12:09:39 +0800 Subject: [PATCH 22/27] ddl: forbid recover/flashback temporary tables (#24518) --- executor/ddl.go | 7 ++++ executor/executor_test.go | 71 ++++++++++++++++++++++----------------- 2 files changed, 48 insertions(+), 30 deletions(-) diff --git a/executor/ddl.go b/executor/ddl.go index 2f10555d21e1e..5058704ee2f9e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -26,12 +26,14 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -563,6 +565,11 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J if tableInfo == nil || jobInfo == nil { return nil, nil, errors.Errorf("Can't find dropped/truncated table: %v in DDL history jobs", tableName.Name) } + // Dropping local temporary tables won't appear in DDL jobs. + if tableInfo.TempTableType == model.TempTableGlobal { + msg := mysql.Message("Recover/flashback table is not supported on temporary tables", nil) + return nil, nil, dbterror.ClassDDL.NewStdErr(errno.ErrUnsupportedDDLOperation, msg) + } return jobInfo, tableInfo, nil } diff --git a/executor/executor_test.go b/executor/executor_test.go index e69b956f8d82b..3b168636606ed 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5697,24 +5697,15 @@ func (s *testRecoverTable) TearDownSuite(c *C) { s.dom.Close() } -func (s *testRecoverTable) TestRecoverTable(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test_recover") - tk.MustExec("use test_recover") - tk.MustExec("drop table if exists t_recover") - tk.MustExec("create table t_recover (a int);") - defer func(originGC bool) { +func (s *testRecoverTable) mockGC(tk *testkit.TestKit) (string, string, string, func()) { + originGC := ddl.IsEmulatorGCEnable() + resetGC := func() { if originGC { ddl.EmulatorGCEnable() } else { ddl.EmulatorGCDisable() } - }(ddl.IsEmulatorGCEnable()) + } // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. @@ -5727,6 +5718,23 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { UPDATE variable_value = '%[1]s'` // clear GC variables first. tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC +} + +func (s *testRecoverTable) TestRecoverTable(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") + c.Assert(err, IsNil) + }() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_recover") + tk.MustExec("use test_recover") + tk.MustExec("drop table if exists t_recover") + tk.MustExec("create table t_recover (a int);") + + timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() tk.MustExec("insert into t_recover values (1),(2),(3)") tk.MustExec("drop table t_recover") @@ -5819,24 +5827,10 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustExec("use test_flashback") tk.MustExec("drop table if exists t_flashback") tk.MustExec("create table t_flashback (a int);") - defer func(originGC bool) { - if originGC { - ddl.EmulatorGCEnable() - } else { - ddl.EmulatorGCDisable() - } - }(ddl.IsEmulatorGCEnable()) - // Disable emulator GC. - // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. - ddl.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') - ON DUPLICATE KEY - UPDATE variable_value = '%[1]s'` - // Clear GC variables first. - tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() + // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. @@ -5939,6 +5933,23 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a from t order by a").Check(testkit.Rows("1", "2", "3")) } +func (s *testRecoverTable) TestRecoverTempTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_recover") + tk.MustExec("use test_recover") + tk.MustExec("drop table if exists t_recover") + tk.MustExec("create global temporary table t_recover (a int) on commit delete rows;") + + timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() + // Set GC safe point + tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) + + tk.MustExec("drop table t_recover") + tk.MustGetErrCode("recover table t_recover;", errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("flashback table t_recover;", errno.ErrUnsupportedDDLOperation) +} + func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists ps_text") From acf2e82b5227cea429347a3c9493214f209205f8 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 13 May 2021 12:45:40 +0800 Subject: [PATCH 23/27] executor: fix point_get result on clustered index when new-row-format disabled but new-collation enabled (#24544) --- executor/point_get.go | 3 +++ session/clustered_index_test.go | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/executor/point_get.go b/executor/point_get.go index b0cd700c5c920..c34987b7f0c1d 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -531,6 +531,9 @@ func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expres chk.AppendInt64(schemaColIdx, handle.IntValue()) return true, nil } + if types.NeedRestoredData(col.RetType) { + return false, nil + } // Try to decode common handle. if mysql.HasPriKeyFlag(col.RetType.Flag) { for i, hid := range pkCols { diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index 0f79b1b13fc2e..b7e529f29fe0e 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -665,3 +665,18 @@ func (s *testClusteredSerialSuite) TestPrefixedClusteredIndexUniqueKeyWithNewCol tk.MustExec("admin check table t;") tk.MustExec("drop table t;") } + +func (s *testClusteredSerialSuite) TestClusteredIndexNewCollationWithOldRowFormat(c *C) { + // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) + // but unistore doesn't support old row format. + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Se.GetSessionVars().RowEncoder.Enable = false + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") + tk.MustExec("insert into t2 select 'aBc'") + tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) +} From 2105317479e90af9282638b18a919262f220095a Mon Sep 17 00:00:00 2001 From: mmyj Date: Thu, 13 May 2021 12:57:39 +0800 Subject: [PATCH 24/27] executor: Improve the performance of appending not fixed columns (#20969) --- util/chunk/chunk.go | 4 ++- util/chunk/chunk_test.go | 77 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 1 deletion(-) diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index a4350bd9628e3..e91cff2559d79 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -505,8 +505,10 @@ func (c *Chunk) Append(other *Chunk, begin, end int) { } else { beginOffset, endOffset := src.offsets[begin], src.offsets[end] dst.data = append(dst.data, src.data[beginOffset:endOffset]...) + lastOffset := dst.offsets[len(dst.offsets)-1] for i := begin; i < end; i++ { - dst.offsets = append(dst.offsets, dst.offsets[len(dst.offsets)-1]+src.offsets[i+1]-src.offsets[i]) + lastOffset += src.offsets[i+1] - src.offsets[i] + dst.offsets = append(dst.offsets, lastOffset) } } for i := begin; i < end; i++ { diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 67222328794db..22a6ac5b473dc 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -1179,3 +1179,80 @@ func BenchmarkBatchAppendRows(b *testing.B) { }) } } + +func BenchmarkAppendRows(b *testing.B) { + b.ReportAllocs() + rowChk := newChunk(8, 8, 0, 0) + + for i := 0; i < 4096; i++ { + rowChk.AppendNull(0) + rowChk.AppendInt64(1, 1) + rowChk.AppendString(2, "abcd") + rowChk.AppendBytes(3, []byte("abcd")) + } + + type testCaseConf struct { + batchSize int + } + testCaseConfs := []testCaseConf{ + {batchSize: 2}, + {batchSize: 8}, + {batchSize: 16}, + {batchSize: 100}, + {batchSize: 1000}, + {batchSize: 4000}, + } + + chk := newChunk(8, 8, 0, 0) + for _, conf := range testCaseConfs { + b.ResetTimer() + b.Run(fmt.Sprintf("row-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + for j := 0; j < conf.batchSize; j++ { + chk.AppendRow(rowChk.GetRow(j)) + } + } + }) + b.ResetTimer() + b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + chk.Append(rowChk, 0, conf.batchSize) + } + }) + } +} + +func BenchmarkAppend(b *testing.B) { + b.ReportAllocs() + rowChk := newChunk(0, 0) + + for i := 0; i < 4096; i++ { + rowChk.AppendString(0, "abcd") + rowChk.AppendBytes(1, []byte("abcd")) + } + + type testCaseConf struct { + batchSize int + } + testCaseConfs := []testCaseConf{ + {batchSize: 2}, + {batchSize: 8}, + {batchSize: 16}, + {batchSize: 100}, + {batchSize: 1000}, + {batchSize: 4000}, + } + + chk := newChunk(0, 0) + for _, conf := range testCaseConfs { + b.ResetTimer() + b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + chk.Append(rowChk, 0, conf.batchSize) + } + }) + } +} From c90f48d80d70d455e93cc606e3ef5162dbf566d6 Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 13 May 2021 13:07:39 +0800 Subject: [PATCH 25/27] *: typo fix (#24564) --- config/config.go | 6 +++--- tidb-server/main.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 7d6560b5d6783..664cb5bceaa7d 100644 --- a/config/config.go +++ b/config/config.go @@ -60,9 +60,9 @@ const ( DefHost = "0.0.0.0" // DefStatusHost is the default status host of TiDB DefStatusHost = "0.0.0.0" - // Def TableColumnCountLimit is limit of the number of columns in a table + // DefTableColumnCountLimit is limit of the number of columns in a table DefTableColumnCountLimit = 1017 - // Def TableColumnCountLimit is maximum limitation of the number of columns in a table + // DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table DefMaxOfTableColumnCountLimit = 4096 ) @@ -73,7 +73,7 @@ var ( "tikv": true, "unistore": true, } - // checkTableBeforeDrop enable to execute `admin check table` before `drop table`. + // CheckTableBeforeDrop enable to execute `admin check table` before `drop table`. CheckTableBeforeDrop = false // checkBeforeDropLDFlag is a go build flag. checkBeforeDropLDFlag = "None" diff --git a/tidb-server/main.go b/tidb-server/main.go index 3e2351bf7c352..6429ba960a0cb 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -629,7 +629,7 @@ func setupMetrics() { metrics.TimeJumpBackCounter.Inc() } callBackCount := 0 - sucessCallBack := func() { + successCallBack := func() { callBackCount++ // It is callback by monitor per second, we increase metrics.KeepAliveCounter per 5s. if callBackCount >= 5 { @@ -637,7 +637,7 @@ func setupMetrics() { metrics.KeepAliveCounter.Inc() } } - go systimemon.StartMonitor(time.Now, systimeErrHandler, sucessCallBack) + go systimemon.StartMonitor(time.Now, systimeErrHandler, successCallBack) pushMetric(cfg.Status.MetricsAddr, time.Duration(cfg.Status.MetricsInterval)*time.Second) } From 956149cdbb8359e618240c7f5568ba5af819eb57 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 13 May 2021 13:35:39 +0800 Subject: [PATCH 26/27] planner/core: refresh stale regions in cache for batch cop response (#24457) --- go.mod | 2 +- go.sum | 4 ++-- store/copr/batch_coprocessor.go | 14 ++++++++++++++ store/mockstore/unistore/tikv/server.go | 5 +++++ store/tikv/region_request_test.go | 4 ++++ 5 files changed, 26 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index b2bd664da13f3..c4b668a81dcf1 100644 --- a/go.mod +++ b/go.mod @@ -45,7 +45,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e + github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 diff --git a/go.sum b/go.sum index c42d129ae5b8b..212917e414404 100644 --- a/go.sum +++ b/go.sum @@ -436,8 +436,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e h1:oUMZ6X/Kpaoxfejh9/jQ+4UZ5xk9MRYcouWJ0oXRKNE= -github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c h1:cy87vgUJT0U4JuxC7R14PuwBrabI9fDawYhyKTbjOBQ= +github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 4bec370a9a4d5..bf8d1b7c893e0 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -406,6 +406,20 @@ func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *copro return errors.Trace(err) } + if len(response.RetryRegions) > 0 { + logutil.BgLogger().Info("multiple regions are stale and need to be refreshed", zap.Int("region size", len(response.RetryRegions))) + for idx, retry := range response.RetryRegions { + id := tikv.NewRegionVerID(retry.Id, retry.RegionEpoch.ConfVer, retry.RegionEpoch.Version) + logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) + b.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + if idx >= 10 { + logutil.BgLogger().Info("stale regions are too many, so we omit the rest ones") + break + } + } + return + } + resp := batchCopResponse{ pbResp: response, detail: new(CopRuntimeStats), diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index f571ff4fe963f..adf3049330897 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -845,6 +845,11 @@ func (svr *Server) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpc return &kvrpcpb.ReadIndexResponse{}, nil } +// GetLockWaitInfo implements implements the tikvpb.TikvServer interface. +func (svr *Server) GetLockWaitInfo(ctx context.Context, _ *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return &kvrpcpb.GetLockWaitInfoResponse{}, nil +} + // transaction debugger commands. // MvccGetByKey implements implements the tikvpb.TikvServer interface. diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 9c5172e52f372..8d531ee209a78 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -462,6 +462,10 @@ func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRe return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return nil, errors.New("unreachable") +} + func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { return errors.New("unreachable") } From f2c2fbda476f71bd1341c2c35ba727d0256a4914 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 13 May 2021 14:13:39 +0800 Subject: [PATCH 27/27] binlog: DML on temporary tables do not write binlog (#24570) --- sessionctx/binloginfo/binloginfo_test.go | 33 ++++++++++++++++++++++++ table/tables/tables.go | 3 +++ 2 files changed, 36 insertions(+) diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 2dfca57d73f4c..b0d9ec91888e4 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -698,3 +698,36 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta c.Assert(err, IsNil) return tbl } + +func (s *testBinlogSuite) TestTempTableBinlog(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.Se.GetSessionVars().BinlogClient = s.client + tk.MustExec("begin") + tk.MustExec("drop table if exists temp_table") + ddlQuery := "create global temporary table temp_table(id int) on commit delete rows" + tk.MustExec(ddlQuery) + ok := mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) + + tk.MustExec("insert temp_table value(1)") + tk.MustExec("update temp_table set id=id+1") + tk.MustExec("commit") + prewriteVal := getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 0) + + tk.MustExec("begin") + tk.MustExec("delete from temp_table") + tk.MustExec("commit") + prewriteVal = getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 0) + + ddlQuery = "truncate table temp_table" + tk.MustExec(ddlQuery) + ok = mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) + + ddlQuery = "drop table if exists temp_table" + tk.MustExec(ddlQuery) + ok = mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 74fd2d82f3ef9..a6a4180401d6a 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1414,6 +1414,9 @@ func shouldWriteBinlog(ctx sessionctx.Context, tblInfo *model.TableInfo) bool { if ctx.GetSessionVars().BinlogClient == nil { return false } + if tblInfo.TempTableType != model.TempTableNone { + return false + } return !ctx.GetSessionVars().InRestrictedSQL }