From f53e3c72cc3c83619d70418ff2c8707fe2168932 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Mon, 27 Jun 2022 10:54:38 +0800 Subject: [PATCH 01/28] ddl: for schema-level DDL method parameter is now XXXStmt (#35722) ref pingcap/tidb#35665, close pingcap/tidb#35734 --- ddl/ddl.go | 4 +- ddl/ddl_api.go | 98 ++++++++++++++++--- domain/domain_test.go | 19 +++- executor/ddl.go | 75 +------------- .../realtikvtest/sessiontest/session_test.go | 3 +- 5 files changed, 104 insertions(+), 95 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index e77a9da01e4a9..0ea01e1222eb5 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -101,9 +101,9 @@ var ( // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. type DDL interface { - CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt, placementPolicyRef *model.PolicyRefInfo) error + CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabaseStmt) error AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) error - DropSchema(ctx sessionctx.Context, schema model.CIStr) error + DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) error CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error CreateView(ctx sessionctx.Context, stmt *ast.CreateViewStmt) error DropTable(ctx sessionctx.Context, tableIdent ast.Ident) (err error) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 5a7758a36ba88..01f57c3ce1d18 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -82,21 +82,76 @@ const ( tiflashCheckPendingTablesRetry = 7 ) -func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt, placementPolicyRef *model.PolicyRefInfo) (err error) { - dbInfo := &model.DBInfo{Name: schema} - if charsetInfo != nil { - chs, coll, err := ResolveCharsetCollation(ast.CharsetOpt{Chs: charsetInfo.Chs, Col: charsetInfo.Col}) +func (d *ddl) CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabaseStmt) (err error) { + var placementPolicyRef *model.PolicyRefInfo + sessionVars := ctx.GetSessionVars() + + // If no charset and/or collation is specified use collation_server and character_set_server + charsetOpt := &ast.CharsetOpt{} + if sessionVars.GlobalVarsAccessor != nil { + charsetOpt.Col, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CollationServer) if err != nil { - return errors.Trace(err) + return err + } + charsetOpt.Chs, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CharacterSetServer) + if err != nil { + return err + } + } + + explicitCharset := false + explicitCollation := false + if len(stmt.Options) != 0 { + for _, val := range stmt.Options { + switch val.Tp { + case ast.DatabaseOptionCharset: + charsetOpt.Chs = val.Value + explicitCharset = true + case ast.DatabaseOptionCollate: + charsetOpt.Col = val.Value + explicitCollation = true + case ast.DatabaseOptionPlacementPolicy: + placementPolicyRef = &model.PolicyRefInfo{ + Name: model.NewCIStr(val.Value), + } + } } - dbInfo.Charset = chs - dbInfo.Collate = coll - } else { - dbInfo.Charset, dbInfo.Collate = charset.GetDefaultCharsetAndCollate() } + if charsetOpt.Col != "" { + coll, err := collate.GetCollationByName(charsetOpt.Col) + if err != nil { + return err + } + + // The collation is not valid for the specified character set. + // Try to remove any of them, but not if they are explicitly defined. + if coll.CharsetName != charsetOpt.Chs { + if explicitCollation && !explicitCharset { + // Use the explicitly set collation, not the implicit charset. + charsetOpt.Chs = "" + } + if !explicitCollation && explicitCharset { + // Use the explicitly set charset, not the (session) collation. + charsetOpt.Col = "" + } + } + + } + dbInfo := &model.DBInfo{Name: stmt.Name} + chs, coll, err := ResolveCharsetCollation(ast.CharsetOpt{Chs: charsetOpt.Chs, Col: charsetOpt.Col}) + if err != nil { + return errors.Trace(err) + } + dbInfo.Charset = chs + dbInfo.Collate = coll dbInfo.PlacementPolicyRef = placementPolicyRef - return d.CreateSchemaWithInfo(ctx, dbInfo, OnExistError) + + onExist := OnExistError + if stmt.IfNotExists { + onExist = OnExistIgnore + } + return d.CreateSchemaWithInfo(ctx, dbInfo, onExist) } func (d *ddl) CreateSchemaWithInfo( @@ -147,6 +202,12 @@ func (d *ddl) CreateSchemaWithInfo( err = d.DoDDLJob(ctx, job) err = d.callHookOnChanged(job, err) + + if infoschema.ErrDatabaseExists.Equal(err) && onExist == OnExistIgnore { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return errors.Trace(err) } @@ -520,11 +581,14 @@ func (d *ddl) AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) return nil } -func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) { +func (d *ddl) DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) (err error) { is := d.GetInfoSchemaWithInterceptor(ctx) - old, ok := is.SchemaByName(schema) + old, ok := is.SchemaByName(stmt.Name) if !ok { - return errors.Trace(infoschema.ErrDatabaseNotExists) + if stmt.IfExists { + return nil + } + return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) } job := &model.Job{ SchemaID: old.ID, @@ -537,13 +601,19 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) err = d.DoDDLJob(ctx, job) err = d.callHookOnChanged(job, err) if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) { + if stmt.IfExists { + return nil + } + return infoschema.ErrDatabaseDropExists.GenWithStackByArgs(stmt.Name) + } return errors.Trace(err) } if !config.TableLockEnabled() { return nil } // Clear table locks hold by the session. - tbs := is.SchemaTables(schema) + tbs := is.SchemaTables(stmt.Name) lockTableIDs := make([]int64, 0) for _, tb := range tbs { if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { diff --git a/domain/domain_test.go b/domain/domain_test.go index b5783a2b97013..98776381d1ae8 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -122,13 +122,22 @@ func TestInfo(t *testing.T) { } require.True(t, syncerStarted) - // Make sure loading schema is normal. - cs := &ast.CharsetOpt{ - Chs: "utf8", - Col: "utf8_bin", + stmt := &ast.CreateDatabaseStmt{ + Name: model.NewCIStr("aaa"), + // Make sure loading schema is normal. + Options: []*ast.DatabaseOption{ + { + Tp: ast.DatabaseOptionCharset, + Value: "utf8", + }, + { + Tp: ast.DatabaseOptionCollate, + Value: "utf8_bin", + }, + }, } ctx := mock.NewContext() - require.NoError(t, dom.ddl.CreateSchema(ctx, model.NewCIStr("aaa"), cs, nil)) + require.NoError(t, dom.ddl.CreateSchema(ctx, stmt)) require.NoError(t, dom.Reload()) require.Equal(t, int64(1), dom.InfoSchema().SchemaMetaVersion()) diff --git a/executor/ddl.go b/executor/ddl.go index af8742fb1f901..1553be2299fd7 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/temptable" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" @@ -248,70 +247,7 @@ func (e *DDLExec) executeRenameTable(s *ast.RenameTableStmt) error { } func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error { - var opt *ast.CharsetOpt - var placementPolicyRef *model.PolicyRefInfo - var err error - sessionVars := e.ctx.GetSessionVars() - - // If no charset and/or collation is specified use collation_server and character_set_server - opt = &ast.CharsetOpt{} - if sessionVars.GlobalVarsAccessor != nil { - opt.Col, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CollationServer) - if err != nil { - return err - } - opt.Chs, err = variable.GetSessionOrGlobalSystemVar(sessionVars, variable.CharacterSetServer) - if err != nil { - return err - } - } - - explicitCharset := false - explicitCollation := false - if len(s.Options) != 0 { - for _, val := range s.Options { - switch val.Tp { - case ast.DatabaseOptionCharset: - opt.Chs = val.Value - explicitCharset = true - case ast.DatabaseOptionCollate: - opt.Col = val.Value - explicitCollation = true - case ast.DatabaseOptionPlacementPolicy: - placementPolicyRef = &model.PolicyRefInfo{ - Name: model.NewCIStr(val.Value), - } - } - } - } - - if opt.Col != "" { - coll, err := collate.GetCollationByName(opt.Col) - if err != nil { - return err - } - - // The collation is not valid for the specified character set. - // Try to remove any of them, but not if they are explicitly defined. - if coll.CharsetName != opt.Chs { - if explicitCollation && !explicitCharset { - // Use the explicitly set collation, not the implicit charset. - opt.Chs = "" - } - if !explicitCollation && explicitCharset { - // Use the explicitly set charset, not the (session) collation. - opt.Col = "" - } - } - - } - - err = domain.GetDomain(e.ctx).DDL().CreateSchema(e.ctx, s.Name, opt, placementPolicyRef) - if err != nil { - if infoschema.ErrDatabaseExists.Equal(err) && s.IfNotExists { - err = nil - } - } + err := domain.GetDomain(e.ctx).DDL().CreateSchema(e.ctx, s) return err } @@ -383,14 +319,7 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { return errors.New("Drop 'mysql' database is forbidden") } - err := domain.GetDomain(e.ctx).DDL().DropSchema(e.ctx, dbName) - if infoschema.ErrDatabaseNotExists.Equal(err) { - if s.IfExists { - err = nil - } else { - err = infoschema.ErrDatabaseDropExists.GenWithStackByArgs(s.Name) - } - } + err := domain.GetDomain(e.ctx).DDL().DropSchema(e.ctx, s) sessionVars := e.ctx.GetSessionVars() if err == nil && strings.ToLower(sessionVars.CurrentDB) == dbName.L { sessionVars.CurrentDB = "" diff --git a/tests/realtikvtest/sessiontest/session_test.go b/tests/realtikvtest/sessiontest/session_test.go index 4bc983d94b40f..9262ebe196498 100644 --- a/tests/realtikvtest/sessiontest/session_test.go +++ b/tests/realtikvtest/sessiontest/session_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" @@ -1363,7 +1364,7 @@ func TestDoDDLJobQuit(t *testing.T) { defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/storeCloseInLoop")) }() // this DDL call will enter deadloop before this fix - err = dom.DDL().CreateSchema(se, model.NewCIStr("testschema"), nil, nil) + err = dom.DDL().CreateSchema(se, &ast.CreateDatabaseStmt{Name: model.NewCIStr("testschema")}) require.Equal(t, "context canceled", err.Error()) } From 0998cba23d1d7e13f8095ec579d744a4d943ec8d Mon Sep 17 00:00:00 2001 From: Spade A <71589810+SpadeA-Tang@users.noreply.github.com> Date: Mon, 27 Jun 2022 11:58:38 +0800 Subject: [PATCH 02/28] txn: refactor ts acquisition within build and execute phases (#35376) close pingcap/tidb#35377 --- executor/adapter.go | 58 ++-- executor/batch_point_get.go | 20 +- executor/builder.go | 72 ++--- executor/executor.go | 19 +- executor/point_get.go | 35 ++- executor/trace_test.go | 2 +- sessiontxn/failpoint.go | 18 ++ sessiontxn/isolation/readcommitted.go | 100 +++--- sessiontxn/isolation/readcommitted_test.go | 92 +++++- sessiontxn/isolation/repeatable_read.go | 71 +++-- sessiontxn/isolation/repeatable_read_test.go | 304 +++++++++++++++---- sessiontxn/txn_context_test.go | 2 +- 12 files changed, 566 insertions(+), 227 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index faf894816c702..31a8c8d20150f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -351,7 +351,7 @@ func IsFastPlan(p plannercore.Plan) bool { } // Exec builds an Executor from a plan. If the Executor doesn't return result, -// like the INSERT, UPDATE statements, it executes in this function, if the Executor returns +// like the INSERT, UPDATE statements, it executes in this function. If the Executor returns // result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { defer func() { @@ -708,7 +708,10 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { keys = filterTemporaryTableKeys(sctx.GetSessionVars(), keys) seVars := sctx.GetSessionVars() keys = filterLockTableKeys(seVars.StmtCtx, keys) - lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout, len(keys)) + lockCtx, err := newLockCtx(sctx, seVars.LockWaitTimeout, len(keys)) + if err != nil { + return err + } var lockKeyStats *util.LockKeysDetails ctx = context.WithValue(ctx, util.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() @@ -730,43 +733,18 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } } -// UpdateForUpdateTS updates the ForUpdateTS, if newForUpdateTS is 0, it obtain a new TS from PD. -func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { - txn, err := seCtx.Txn(false) - if err != nil { - return err - } - if !txn.Valid() { - return errors.Trace(kv.ErrInvalidTxn) - } - - // The Oracle serializable isolation is actually SI in pessimistic mode. - // Do not update ForUpdateTS when the user is using the Serializable isolation level. - // It can be used temporarily on the few occasions when an Oracle-like isolation level is needed. - // Support for this does not mean that TiDB supports serializable isolation of MySQL. - // tidb_skip_isolation_level_check should still be disabled by default. - if seCtx.GetSessionVars().IsIsolation(ast.Serializable) { - return nil - } - if newForUpdateTS == 0 { - // Because the ForUpdateTS is used for the snapshot for reading data in DML. - // We can avoid allocating a global TSO here to speed it up by using the local TSO. - version, err := seCtx.GetStore().CurrentVersion(seCtx.GetSessionVars().TxnCtx.TxnScope) - if err != nil { - return err - } - newForUpdateTS = version.Ver - } - seCtx.GetSessionVars().TxnCtx.SetForUpdateTS(newForUpdateTS) - txn.SetOption(kv.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) - return nil -} - // handlePessimisticLockError updates TS and rebuild executor if the err is write conflict. func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error) (_ Executor, err error) { if lockErr == nil { return nil, nil } + failpoint.Inject("assertPessimisticLockErr", func() { + if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { + sessiontxn.AddAssertEntranceForLockError(a.Ctx, "errWriteConflict") + } else if terror.ErrorEqual(kv.ErrKeyExists, lockErr) { + sessiontxn.AddAssertEntranceForLockError(a.Ctx, "errDuplicateKey") + } + }) defer func() { if _, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { @@ -774,7 +752,8 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error } }() - action, err := sessiontxn.GetTxnManager(a.Ctx).OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) + txnManager := sessiontxn.GetTxnManager(a.Ctx) + action, err := txnManager.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) if err != nil { return nil, err } @@ -789,10 +768,17 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, lockErr error a.retryCount++ a.retryStartTime = time.Now() - err = sessiontxn.GetTxnManager(a.Ctx).OnStmtRetry(ctx) + err = txnManager.OnStmtRetry(ctx) if err != nil { return nil, err } + + // Without this line of code, the result will still be correct. But it can ensure that the update time of for update read + // is determined which is beneficial for testing. + if _, err = txnManager.GetStmtForUpdateTS(); err != nil { + return nil, err + } + breakpoint.Inject(a.Ctx, sessiontxn.BreakPointOnStmtRetryAfterLockError) e, err := a.buildExecutor() diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 0ce745d172e4a..b5eb68a8b12de 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -56,7 +56,6 @@ type BatchPointGetExec struct { singlePart bool partTblID int64 idxVals [][]types.Datum - startTS uint64 readReplicaScope string isStaleness bool snapshotTS uint64 @@ -97,13 +96,9 @@ func (e *BatchPointGetExec) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *BatchPointGetExec) Open(context.Context) error { - e.snapshotTS = e.startTS sessVars := e.ctx.GetSessionVars() txnCtx := sessVars.TxnCtx stmtCtx := sessVars.StmtCtx - if e.lock { - e.snapshotTS = txnCtx.GetForUpdateTS() - } txn, err := e.ctx.Txn(false) if err != nil { return err @@ -111,8 +106,8 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.txn = txn var snapshot kv.Snapshot if txn.Valid() && txnCtx.StartTS == txnCtx.GetForUpdateTS() && txnCtx.StartTS == e.snapshotTS { - // We can safely reuse the transaction snapshot if startTS is equal to forUpdateTS. - // The snapshot may contains cache that can reduce RPC call. + // We can safely reuse the transaction snapshot if snapshotTS is equal to forUpdateTS. + // The snapshot may contain cache that can reduce RPC call. snapshot = txn.GetSnapshot() } else { snapshot = e.ctx.GetSnapshotWithTS(e.snapshotTS) @@ -540,13 +535,16 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { } // LockKeys locks the keys for pessimistic transaction. -func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { - txnCtx := seCtx.GetSessionVars().TxnCtx - lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime, len(keys)) +func LockKeys(ctx context.Context, sctx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { + txnCtx := sctx.GetSessionVars().TxnCtx + lctx, err := newLockCtx(sctx, lockWaitTime, len(keys)) + if err != nil { + return err + } if txnCtx.IsPessimistic { lctx.InitReturnValues(len(keys)) } - err := doLockKeys(ctx, seCtx, lctx, keys...) + err = doLockKeys(ctx, sctx, lctx, keys...) if err != nil { return err } diff --git a/executor/builder.go b/executor/builder.go index 8a44c09aaf033..73a4995696de0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -657,11 +657,11 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor defer func() { b.inSelectLockStmt = false }() } b.hasLock = true - if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { + + // Build 'select for update' using the 'for update' ts. + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - // Build 'select for update' using the 'for update' ts. - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() src := b.build(v.Children()[0]) if b.err != nil { @@ -865,14 +865,11 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - if v.SelectPlan != nil { - // Try to update the forUpdateTS for insert/replace into select statements. - // Set the selectPlan parameter to nil to make it always update the forUpdateTS. - if b.err = b.updateForUpdateTSIfNeeded(nil); b.err != nil { - return nil - } + + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { + return nil } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + selectExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2116,10 +2113,11 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } } - if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { + + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2173,10 +2171,11 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { for _, info := range v.TblColPosInfos { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { + + if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { return nil } - b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2192,34 +2191,6 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { return deleteExec } -// updateForUpdateTSIfNeeded updates the ForUpdateTS for a pessimistic transaction if needed. -// PointGet executor will get conflict error if the ForUpdateTS is older than the latest commitTS, -// so we don't need to update now for better latency. -func (b *executorBuilder) updateForUpdateTSIfNeeded(selectPlan plannercore.PhysicalPlan) error { - txnCtx := b.ctx.GetSessionVars().TxnCtx - if !txnCtx.IsPessimistic { - return nil - } - if _, ok := selectPlan.(*plannercore.PointGetPlan); ok { - return nil - } - // Activate the invalid txn, use the txn startTS as newForUpdateTS - txn, err := b.ctx.Txn(false) - if err != nil { - return err - } - if !txn.Valid() { - _, err := b.ctx.Txn(true) - if err != nil { - return err - } - return nil - } - // GetStmtForUpdateTS will auto update the for update ts if necessary - _, err = sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS() - return err -} - func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O} _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) @@ -4663,18 +4634,26 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan return nil } - startTS, err := b.getSnapshotTS() + if plan.Lock && !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { + b.inSelectLockStmt = false + }() + } + + snapshotTS, err := b.getSnapshotTS() if err != nil { b.err = err return nil } + decoder := NewRowDecoder(b.ctx, plan.Schema(), plan.TblInfo) e := &BatchPointGetExec{ baseExecutor: newBaseExecutor(b.ctx, plan.Schema(), plan.ID()), tblInfo: plan.TblInfo, idxInfo: plan.IndexInfo, rowDecoder: decoder, - startTS: startTS, + snapshotTS: snapshotTS, readReplicaScope: b.readReplicaScope, isStaleness: b.isStaleness, keepOrder: plan.KeepOrder, @@ -4687,9 +4666,11 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan partTblID: plan.PartTblID, columns: plan.Columns, } + if plan.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(plan.TblInfo, startTS) + e.cacheTable = b.getCacheTable(plan.TblInfo, snapshotTS) } + if plan.TblInfo.TempTableType != model.TempTableNone { // Temporary table should not do any lock operations e.lock = false @@ -4699,6 +4680,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan if e.lock { b.hasLock = true } + var capacity int if plan.IndexInfo != nil && !isCommonHandleRead(plan.TblInfo, plan.IndexInfo) { e.idxVals = plan.IndexValues diff --git a/executor/executor.go b/executor/executor.go index aa71929859898..d2b726f24adbb 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -1042,12 +1043,20 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { for id := range e.tblID2Handle { e.updateDeltaForTableID(id) } - - return doLockKeys(ctx, e.ctx, newLockCtx(e.ctx.GetSessionVars(), lockWaitTime, len(e.keys)), e.keys...) + lockCtx, err := newLockCtx(e.ctx, lockWaitTime, len(e.keys)) + if err != nil { + return err + } + return doLockKeys(ctx, e.ctx, lockCtx, e.keys...) } -func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) *tikvstore.LockCtx { - lockCtx := tikvstore.NewLockCtx(seVars.TxnCtx.GetForUpdateTS(), lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) +func newLockCtx(sctx sessionctx.Context, lockWaitTime int64, numKeys int) (*tikvstore.LockCtx, error) { + seVars := sctx.GetSessionVars() + forUpdateTS, err := sessiontxn.GetTxnManager(sctx).GetStmtForUpdateTS() + if err != nil { + return nil, err + } + lockCtx := tikvstore.NewLockCtx(forUpdateTS, lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) lockCtx.Killed = &seVars.Killed lockCtx.PessimisticLockWaited = &seVars.StmtCtx.PessimisticLockWaited lockCtx.LockKeysDuration = &seVars.StmtCtx.LockKeysDuration @@ -1082,7 +1091,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) * if lockCtx.ForUpdateTS > 0 && seVars.AssertionLevel != variable.AssertionLevelOff { lockCtx.InitCheckExistence(numKeys) } - return lockCtx + return lockCtx, nil } // doLockKeys is the main entry for pessimistic lock keys diff --git a/executor/point_get.go b/executor/point_get.go index f33ba20b5dd5a..1b4d6666663b5 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -49,11 +49,19 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { return nil } - startTS, err := b.getSnapshotTS() + if p.Lock && !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { + b.inSelectLockStmt = false + }() + } + + snapshotTS, err := b.getSnapshotTS() if err != nil { b.err = err return nil } + e := &PointGetExecutor{ baseExecutor: newBaseExecutor(b.ctx, p.Schema(), p.ID()), readReplicaScope: b.readReplicaScope, @@ -61,14 +69,17 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { } if p.TblInfo.TableCacheStatusType == model.TableCacheStatusEnable { - e.cacheTable = b.getCacheTable(p.TblInfo, startTS) + e.cacheTable = b.getCacheTable(p.TblInfo, snapshotTS) } + e.base().initCap = 1 e.base().maxChunkSize = 1 - e.Init(p, startTS) + e.Init(p, snapshotTS) + if e.lock { b.hasLock = true } + return e } @@ -83,7 +94,7 @@ type PointGetExecutor struct { idxKey kv.Key handleVal []byte idxVals []types.Datum - startTS uint64 + snapshotTS uint64 readReplicaScope string isStaleness bool txn kv.Transaction @@ -106,13 +117,13 @@ type PointGetExecutor struct { } // Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field -func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, startTs uint64) { +func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan, snapshotTS uint64) { decoder := NewRowDecoder(e.ctx, p.Schema(), p.TblInfo) e.tblInfo = p.TblInfo e.handle = p.Handle e.idxInfo = p.IndexInfo e.idxVals = p.IndexValues - e.startTS = startTs + e.snapshotTS = snapshotTS e.done = false if e.tblInfo.TempTableType == model.TempTableNone { e.lock = p.Lock @@ -142,10 +153,7 @@ func (e *PointGetExecutor) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *PointGetExecutor) Open(context.Context) error { txnCtx := e.ctx.GetSessionVars().TxnCtx - snapshotTS := e.startTS - if e.lock { - snapshotTS = txnCtx.GetForUpdateTS() - } + snapshotTS := e.snapshotTS var err error e.txn, err = e.ctx.Txn(false) if err != nil { @@ -381,9 +389,12 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro } if e.lock { seVars := e.ctx.GetSessionVars() - lockCtx := newLockCtx(seVars, e.lockWaitTime, 1) + lockCtx, err := newLockCtx(e.ctx, e.lockWaitTime, 1) + if err != nil { + return err + } lockCtx.InitReturnValues(1) - err := doLockKeys(ctx, e.ctx, lockCtx, key) + err = doLockKeys(ctx, e.ctx, lockCtx, key) if err != nil { return err } diff --git a/executor/trace_test.go b/executor/trace_test.go index f8e8e91ddebd7..9b448670cc39a 100644 --- a/executor/trace_test.go +++ b/executor/trace_test.go @@ -33,7 +33,7 @@ func TestTraceExec(t *testing.T) { require.GreaterOrEqual(t, len(rows), 1) // +---------------------------+-----------------+------------+ - // | operation | startTS | duration | + // | operation | snapshotTS | duration | // +---------------------------+-----------------+------------+ // | session.getTxnFuture | 22:08:38.247834 | 78.909µs | // | ├─session.Execute | 22:08:38.247829 | 1.478487ms | diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go index d33984649b371..b41be21165908 100644 --- a/sessiontxn/failpoint.go +++ b/sessiontxn/failpoint.go @@ -43,6 +43,10 @@ var BreakPointBeforeExecutorFirstRun = "beforeExecutorFirstRun" // Only for test var BreakPointOnStmtRetryAfterLockError = "lockErrorAndThenOnStmtRetryCalled" +// AssertLockErr is used to record the lock errors we encountered +// Only for test +var AssertLockErr stringutil.StringerStr = "assertLockError" + // RecordAssert is used only for test func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { records, ok := sctx.Value(AssertRecordsKey).(map[string]interface{}) @@ -94,6 +98,20 @@ func AssertTxnManagerReadTS(sctx sessionctx.Context, expected uint64) { } } +// AddAssertEntranceForLockError is used only for test +func AddAssertEntranceForLockError(sctx sessionctx.Context, name string) { + records, ok := sctx.Value(AssertLockErr).(map[string]int) + if !ok { + records = make(map[string]int) + sctx.SetValue(AssertLockErr, records) + } + if v, ok := records[name]; ok { + records[name] = v + 1 + } else { + records[name] = 1 + } +} + // ExecTestHook is used only for test. It consumes hookKey in session wait do what it gets from it. func ExecTestHook(sctx sessionctx.Context, hookKey fmt.Stringer) { c := sctx.Value(hookKey) diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 8a409000d7049..5fb316b59f8bf 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/terror" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -31,20 +32,15 @@ import ( ) type stmtState struct { - stmtTS uint64 - stmtTSFuture oracle.Future - stmtUseStartTS bool - onNextRetryOrStmt func() error + stmtTS uint64 + stmtTSFuture oracle.Future + stmtUseStartTS bool } func (s *stmtState) prepareStmt(useStartTS bool) error { - onNextStmt := s.onNextRetryOrStmt *s = stmtState{ stmtUseStartTS: useStartTS, } - if onNextStmt != nil { - return onNextStmt() - } return nil } @@ -52,7 +48,9 @@ func (s *stmtState) prepareStmt(useStartTS bool) error { type PessimisticRCTxnContextProvider struct { baseTxnContextProvider stmtState - availableRCCheckTS uint64 + latestOracleTS uint64 + // latestOracleTSValid shows whether we have already fetched a ts from pd and whether the ts we fetched is still valid. + latestOracleTSValid bool } // NewPessimisticRCTxnContextProvider returns a new PessimisticRCTxnContextProvider @@ -65,12 +63,14 @@ func NewPessimisticRCTxnContextProvider(sctx sessionctx.Context, causalConsisten txnCtx.IsPessimistic = true txnCtx.Isolation = ast.ReadCommitted }, - onTxnActive: func(txn kv.Transaction) { - txn.SetOption(kv.Pessimistic, true) - }, }, } + provider.onTxnActive = func(txn kv.Transaction) { + txn.SetOption(kv.Pessimistic, true) + provider.latestOracleTS = txn.StartTS() + provider.latestOracleTSValid = true + } provider.getStmtReadTSFunc = provider.getStmtTS provider.getStmtForUpdateTSFunc = provider.getStmtTS return provider @@ -86,9 +86,6 @@ func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context) error // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { - // Invalid rc check for next statement or retry when error occurs - p.availableRCCheckTS = 0 - switch point { case sessiontxn.StmtErrAfterQuery: return p.handleAfterQueryError(err) @@ -117,15 +114,30 @@ func (p *PessimisticRCTxnContextProvider) prepareStmtTS() { switch { case p.stmtUseStartTS: stmtTSFuture = sessiontxn.FuncFuture(p.getTxnStartTS) - case p.availableRCCheckTS != 0 && sessVars.StmtCtx.RCCheckTS: - stmtTSFuture = sessiontxn.ConstantFuture(p.availableRCCheckTS) + case p.latestOracleTSValid && sessVars.StmtCtx.RCCheckTS: + stmtTSFuture = sessiontxn.ConstantFuture(p.latestOracleTS) default: - stmtTSFuture = sessiontxn.NewOracleFuture(p.ctx, p.sctx, sessVars.TxnCtx.TxnScope) + stmtTSFuture = p.getOracleFuture() } p.stmtTSFuture = stmtTSFuture } +func (p *PessimisticRCTxnContextProvider) getOracleFuture() sessiontxn.FuncFuture { + txnCtx := p.sctx.GetSessionVars().TxnCtx + future := sessiontxn.NewOracleFuture(p.ctx, p.sctx, txnCtx.TxnScope) + return func() (ts uint64, err error) { + if ts, err = future.Wait(); err != nil { + return + } + txnCtx.SetForUpdateTS(ts) + ts = txnCtx.GetForUpdateTS() + p.latestOracleTS = ts + p.latestOracleTSValid = true + return + } +} + func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { if p.stmtTS != 0 { return p.stmtTS, nil @@ -141,13 +153,8 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { return 0, err } - // forUpdateTS should exactly equal to the read ts - txnCtx := p.sctx.GetSessionVars().TxnCtx - txnCtx.SetForUpdateTS(ts) txn.SetOption(kv.SnapshotTS, ts) - p.stmtTS = ts - p.availableRCCheckTS = ts return } @@ -155,16 +162,18 @@ func (p *PessimisticRCTxnContextProvider) getStmtTS() (ts uint64, err error) { // At this point the query will be retried from the beginning. func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) (sessiontxn.StmtErrorAction, error) { sessVars := p.sctx.GetSessionVars() - if sessVars.StmtCtx.RCCheckTS && errors.ErrorEqual(queryErr, kv.ErrWriteConflict) { - logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", - zap.String("sql", sessVars.StmtCtx.OriginalSQL)) - return sessiontxn.RetryReady() + if !errors.ErrorEqual(queryErr, kv.ErrWriteConflict) || !sessVars.StmtCtx.RCCheckTS { + return sessiontxn.NoIdea() } - return sessiontxn.NoIdea() + p.latestOracleTSValid = false + logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", + zap.String("sql", sessVars.StmtCtx.OriginalSQL)) + return sessiontxn.RetryReady() } func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { + p.latestOracleTSValid = false txnCtx := p.sctx.GetSessionVars().TxnCtx retryable := false if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok && deadlock.IsRetryable { @@ -182,16 +191,9 @@ func (p *PessimisticRCTxnContextProvider) handleAfterPessimisticLockError(lockEr retryable = true } - // force refresh ts in next retry or statement when lock error occurs - p.onNextRetryOrStmt = func() error { - _, err := p.getStmtTS() - return err - } - if retryable { return sessiontxn.RetryReady() } - return sessiontxn.ErrorAction(lockErr) } @@ -207,3 +209,31 @@ func (p *PessimisticRCTxnContextProvider) AdviseWarmup() error { p.prepareStmtTS() return nil } + +// AdviseOptimizeWithPlan in RC covers much fewer cases compared with pessimistic repeatable read. +// We only optimize with insert operator with no selection in that we do not fetch latest ts immediately. +// We only update ts if write conflict is incurred. +func (p *PessimisticRCTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { + if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { + return nil + } + + if p.stmtUseStartTS || !p.latestOracleTSValid { + return nil + } + + plan, ok := val.(plannercore.Plan) + if !ok { + return nil + } + + if execute, ok := plan.(*plannercore.Execute); ok { + plan = execute.Plan + } + + if v, ok := plan.(*plannercore.Insert); ok && v.SelectPlan == nil { + p.stmtTSFuture = sessiontxn.ConstantFuture(p.latestOracleTS) + } + + return nil +} diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index d867cc689ca28..5c747eba4fa2c 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" @@ -53,13 +54,13 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.NoError(t, err) forUpdateStmt := stmts[0] - compareTS := getOracleTS(t, se) + compareTS := se.GetSessionVars().TxnCtx.StartTS // first ts should request from tso require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) require.NoError(t, provider.OnStmtStart(context.TODO())) ts, err := provider.GetStmtReadTS() require.NoError(t, err) - require.Greater(t, ts, compareTS) + require.Equal(t, ts, compareTS) rcCheckTS := ts // second ts should reuse first ts @@ -101,14 +102,11 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, errors.New("err")) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) - compareTS = getOracleTS(t, se) - require.Greater(t, compareTS, rcCheckTS) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) require.NoError(t, provider.OnStmtStart(context.TODO())) ts, err = provider.GetStmtReadTS() require.NoError(t, err) - require.Greater(t, ts, compareTS) - rcCheckTS = ts + require.Equal(t, rcCheckTS, ts) // `StmtErrAfterPessimisticLock` will still disable rc check require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) @@ -381,6 +379,88 @@ func TestTidbSnapshotVarInRC(t *testing.T) { } } +func TestConflictErrorsInRC(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("set tx_isolation='READ-COMMITTED'") + + // Test for insert + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 2)") + se.SetValue(sessiontxn.AssertLockErr, nil) + _, err := tk.Exec("insert into t values (1, 1), (2, 2)") + require.Error(t, err) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + for _, name := range errorsInInsert { + require.Equal(t, records[name], 1) + } + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("rollback") + + // Test for delete + tk.MustExec("truncate t") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (3, 1)") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where v = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) + tk.MustExec("commit") + + // Unlike RR, in RC, we will always fetch the latest ts. So write conflict will not be happened + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set id = 1 where id = 2") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where id = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t for update").Check(testkit.Rows()) + + tk.MustExec("rollback") + + // Test for update + tk.MustExec("truncate t") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10") + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("update t set v = v + 10") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) + tk.MustExec("commit") + + // Unlike RR, in RC, we will always fetch the latest ts. So write conflict will not be happened + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustExec("update t set v = v + 10 where id = 1") + _, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) + + tk.MustExec("rollback") + + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + func activeRCTxnAssert(t *testing.T, sctx sessionctx.Context, inTxn bool) *txnAssert[*isolation.PessimisticRCTxnContextProvider] { return &txnAssert[*isolation.PessimisticRCTxnContextProvider]{ sctx: sctx, diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index ef678f40c614c..571d2754be9a3 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -35,10 +35,11 @@ type PessimisticRRTxnContextProvider struct { baseTxnContextProvider // Used for ForUpdateRead statement - forUpdateTS uint64 + forUpdateTS uint64 + latestForUpdateTS uint64 // It may decide whether to update forUpdateTs when calling provider's getForUpdateTs // See more details in the comments of optimizeWithPlan - followingOperatorIsPointGetForUpdate bool + optimizeForNotFetchingLatestTS bool } // NewPessimisticRRTxnContextProvider returns a new PessimisticRRTxnContextProvider @@ -73,7 +74,7 @@ func (p *PessimisticRRTxnContextProvider) getForUpdateTs() (ts uint64, err error return 0, err } - if p.followingOperatorIsPointGetForUpdate { + if p.optimizeForNotFetchingLatestTS { p.forUpdateTS = p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS() return p.forUpdateTS, nil } @@ -114,7 +115,8 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { } sctx.GetSessionVars().TxnCtx.SetForUpdateTS(version.Ver) - txn.SetOption(kv.SnapshotTS, sctx.GetSessionVars().TxnCtx.GetForUpdateTS()) + p.latestForUpdateTS = version.Ver + txn.SetOption(kv.SnapshotTS, version.Ver) return nil } @@ -126,7 +128,7 @@ func (p *PessimisticRRTxnContextProvider) OnStmtStart(ctx context.Context) error } p.forUpdateTS = 0 - p.followingOperatorIsPointGetForUpdate = false + p.optimizeForNotFetchingLatestTS = false return nil } @@ -137,15 +139,14 @@ func (p *PessimisticRRTxnContextProvider) OnStmtRetry(ctx context.Context) (err return err } - txnCtxForUpdateTS := p.sctx.GetSessionVars().TxnCtx.GetForUpdateTS() // If TxnCtx.forUpdateTS is updated in OnStmtErrorForNextAction, we assign the value to the provider - if txnCtxForUpdateTS > p.forUpdateTS { - p.forUpdateTS = txnCtxForUpdateTS + if p.latestForUpdateTS > p.forUpdateTS { + p.forUpdateTS = p.latestForUpdateTS } else { p.forUpdateTS = 0 } - p.followingOperatorIsPointGetForUpdate = false + p.optimizeForNotFetchingLatestTS = false return nil } @@ -165,6 +166,8 @@ func (p *PessimisticRRTxnContextProvider) OnStmtErrorForNextAction(point session // We expect that the data that the point get acquires has not been changed. // Benefit: Save the cost of acquiring ts from PD. // Drawbacks: If the data has been changed since the ts we used, we need to retry. +// One exception is insert operation, when it has no select plan, we do not fetch the latest ts immediately. We only update ts +// if write conflict is incurred. func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{}) (err error) { if p.isTidbSnapshotEnabled() || p.isBeginStmtWithStaleRead() { return nil @@ -179,24 +182,44 @@ func (p *PessimisticRRTxnContextProvider) AdviseOptimizeWithPlan(val interface{} plan = execute.Plan } - mayOptimizeForPointGet := false - if v, ok := plan.(*plannercore.PhysicalLock); ok { - if _, ok := v.Children()[0].(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true - } - } else if v, ok := plan.(*plannercore.Update); ok { - if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + p.optimizeForNotFetchingLatestTS = notNeedGetLatestTSFromPD(plan, false) + + return nil +} + +// notNeedGetLatestTSFromPD searches for optimization condition recursively +// Note: For point get and batch point get (name it plan), if one of the ancestor node is update/delete/physicalLock, +// we should check whether the plan.Lock is true or false. See comments in needNotToBeOptimized. +// inLockOrWriteStmt = true means one of the ancestor node is update/delete/physicalLock. +func notNeedGetLatestTSFromPD(plan plannercore.Plan, inLockOrWriteStmt bool) bool { + switch v := plan.(type) { + case *plannercore.PointGetPlan: + // We do not optimize the point get/ batch point get if plan.lock = false and inLockOrWriteStmt = true. + // Theoretically, the plan.lock should be true if the flag is true. But due to the bug describing in Issue35524, + // the plan.lock can be false in the case of inLockOrWriteStmt being true. In this case, optimization here can lead to different results + // which cannot be accepted as AdviseOptimizeWithPlan cannot change results. + return !inLockOrWriteStmt || v.Lock + case *plannercore.BatchPointGetPlan: + return !inLockOrWriteStmt || v.Lock + case plannercore.PhysicalPlan: + if len(v.Children()) == 0 { + return false } - } else if v, ok := plan.(*plannercore.Delete); ok { - if _, ok := v.SelectPlan.(*plannercore.PointGetPlan); ok { - mayOptimizeForPointGet = true + _, isPhysicalLock := v.(*plannercore.PhysicalLock) + for _, p := range v.Children() { + if !notNeedGetLatestTSFromPD(p, isPhysicalLock || inLockOrWriteStmt) { + return false + } } + return true + case *plannercore.Update: + return notNeedGetLatestTSFromPD(v.SelectPlan, true) + case *plannercore.Delete: + return notNeedGetLatestTSFromPD(v.SelectPlan, true) + case *plannercore.Insert: + return v.SelectPlan == nil } - - p.followingOperatorIsPointGetForUpdate = mayOptimizeForPointGet - - return nil + return false } func (p *PessimisticRRTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index dfeed73e9af33..c60c1c3da560d 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" @@ -344,60 +345,89 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { tk.MustExec("insert into t values (1,1), (2,2)") se := tk.Session() provider := initializeRepeatableReadProvider(t, tk, true) - forUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() + lastFetchedForUpdateTS := se.GetSessionVars().TxnCtx.GetForUpdateTS() txnManager := sessiontxn.GetTxnManager(se) - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err := parser.New().ParseOneStmt("delete from t where id = 1", "", "") - require.NoError(t, err) - compareTs := getOracleTS(t, se) - compiler := executor.Compiler{Ctx: se} - execStmt, err := compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err := provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, compareTs, ts) - require.Equal(t, ts, forUpdateTS) + type testStruct struct { + sql string + shouldOptimize bool + } - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err = parser.New().ParseOneStmt("update t set v = v + 10 where id = 1", "", "") - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) + cases := []testStruct{ + { + "delete from t where id = 1", + true, + }, + { + "update t set v = v + 10 where id = 1", + true, + }, + { + "select * from (select * from t where id = 1 for update) as t1 for update", + true, + }, + { + "select * from t where id = 1 for update", + true, + }, + { + "select * from t where id = 1 or id = 2 for update", + true, + }, + { + "select * from t for update", + false, + }, + } - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err = parser.New().ParseOneStmt("select * from (select * from t where id = 1 for update) as t1 for update", "", "") - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Equal(t, ts, forUpdateTS) + var stmt ast.StmtNode + var err error + var execStmt *executor.ExecStmt + var compiler executor.Compiler + var ts, compareTS uint64 + var action sessiontxn.StmtErrorAction - // Now, test for one that does not use the optimization - require.NoError(t, txnManager.OnStmtStart(context.TODO())) - stmt, err = parser.New().ParseOneStmt("select * from t for update", "", "") - compareTs = getOracleTS(t, se) - require.NoError(t, err) - compiler = executor.Compiler{Ctx: se} - execStmt, err = compiler.Compile(context.TODO(), stmt) - require.NoError(t, err) - err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) - require.NoError(t, err) - ts, err = provider.GetStmtForUpdateTS() - require.NoError(t, err) - require.Greater(t, ts, compareTs) + for _, c := range cases { + compareTS = getOracleTS(t, se) + + require.NoError(t, txnManager.OnStmtStart(context.TODO())) + stmt, err = parser.New().ParseOneStmt(c.sql, "", "") + require.NoError(t, err) + + err = provider.OnStmtStart(context.TODO()) + require.NoError(t, err) + + compiler = executor.Compiler{Ctx: se} + execStmt, err = compiler.Compile(context.TODO(), stmt) + require.NoError(t, err) + + err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) + require.NoError(t, err) + + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + + if c.shouldOptimize { + require.Greater(t, compareTS, ts) + require.Equal(t, ts, lastFetchedForUpdateTS) + } else { + require.Greater(t, ts, compareTS) + } + + // retry + if c.shouldOptimize { + action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, kv.ErrWriteConflict) + require.NoError(t, err) + require.Equal(t, sessiontxn.StmtActionRetryReady, action) + err = provider.OnStmtRetry(context.TODO()) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Greater(t, ts, compareTS) + + lastFetchedForUpdateTS = ts + } + } // Test use startTS after optimize when autocommit=0 activeAssert := activePessimisticRRAssert(t, tk.Session(), true) @@ -415,7 +445,7 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.Equal(t, tk.Session().GetSessionVars().TxnCtx.StartTS, ts) // Test still fetch for update ts after optimize when autocommit=0 - compareTs = getOracleTS(t, se) + compareTS = getOracleTS(t, se) activeAssert = activePessimisticRRAssert(t, tk.Session(), true) provider = initializeRepeatableReadProvider(t, tk, false) require.NoError(t, txnManager.OnStmtStart(context.TODO())) @@ -427,7 +457,179 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { require.NoError(t, err) ts, err = provider.GetStmtForUpdateTS() require.NoError(t, err) - require.Greater(t, ts, compareTs) + require.Greater(t, ts, compareTS) +} + +var errorsInInsert = []string{ + "errWriteConflict", + "errDuplicateKey", +} + +func TestConflictErrorInInsertInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (1, 2)") + se.SetValue(sessiontxn.AssertLockErr, nil) + _, err := tk.Exec("insert into t values (1, 1), (2, 2)") + require.Error(t, err) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + for _, name := range errorsInInsert { + require.Equal(t, records[name], 1) + } + + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestConflictErrorInPointGetForUpdateInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id = 1 for update").Check(testkit.Rows("1 11")) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustExec("commit") + + // batch point get + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id = 1 or id = 2 for update").Check(testkit.Rows("1 21", "2 2")) + records, ok = se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustExec("commit") + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +// Delete should get the latest ts and thus does not incur write conflict +func TestConflictErrorInDeleteInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("insert into t values (3, 1)") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where v = 1") + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("2 2")) + tk.MustExec("commit") + + tk.MustExec("begin pessimistic") + // However, if sub select in delete is point get, we will incur one write conflict + tk2.MustExec("update t set id = 1 where id = 2") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("delete from t where id = 1") + + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustQuery("select * from t for update").Check(testkit.Rows()) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestConflictErrorInUpdateInRR(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1), (2, 2)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustExec("update t set v = v + 10") + _, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.False(t, ok) + tk.MustQuery("select * from t").Check(testkit.Rows("1 21", "2 22")) + tk.MustExec("commit") + + tk.MustExec("begin pessimistic") + // However, if the sub select plan is point get, we should incur one write conflict + tk2.MustExec("update t set v = v + 10 where id = 1") + tk.MustExec("update t set v = v + 10 where id = 1") + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + tk.MustQuery("select * from t for update").Check(testkit.Rows("1 41", "2 22")) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) +} + +func TestConflictErrorInOtherQueryContainingPointGet(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertPessimisticLockErr", "return")) + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + se := tk.Session() + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk.MustExec("insert into t values (1, 1)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("update t set v = v + 10 where id = 1") + se.SetValue(sessiontxn.AssertLockErr, nil) + tk.MustQuery("select * from t where id=1 and v > 1 for update").Check(testkit.Rows("1 11")) + records, ok := se.Value(sessiontxn.AssertLockErr).(map[string]int) + require.True(t, ok) + require.Equal(t, records["errWriteConflict"], 1) + + tk.MustExec("rollback") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertPessimisticLockErr")) } func activePessimisticRRAssert(t *testing.T, sctx sessionctx.Context, diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index d3e1f32124575..e247a14c86f2b 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -445,7 +445,7 @@ func TestTxnContextForHistoricalRead(t *testing.T) { }) doWithCheckPath(t, se, normalPathRecords, func() { - tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) }) tk.MustExec("rollback") From 31c92c67bc6c2b5d8e38a90f48d2d72aac8525f2 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Mon, 27 Jun 2022 12:20:39 +0800 Subject: [PATCH 03/28] sessionctx: support encoding and decoding statement context (#35688) close pingcap/tidb#35664 --- executor/executor.go | 9 +- session/session.go | 9 +- sessionctx/sessionstates/session_states.go | 4 + .../sessionstates/session_states_test.go | 119 ++++++++++++++++++ sessionctx/stmtctx/stmtctx.go | 49 ++++++++ sessionctx/stmtctx/stmtctx_test.go | 43 +++++++ sessionctx/variable/session.go | 10 ++ 7 files changed, 237 insertions(+), 6 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index d2b726f24adbb..491f3b2b4e27a 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1928,7 +1928,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.IgnoreTruncate = true sc.IgnoreZeroInDate = true sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() - if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors { + if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors || stmt.Tp == ast.ShowSessionStates { sc.InShowWarning = true sc.SetWarnings(vars.StmtCtx.GetWarnings()) } @@ -1936,6 +1936,11 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.IgnoreTruncate = false sc.IgnoreZeroInDate = true sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() + case *ast.SetSessionStatesStmt: + sc.InSetSessionStatesStmt = true + sc.IgnoreTruncate = true + sc.IgnoreZeroInDate = true + sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() default: sc.IgnoreTruncate = true sc.IgnoreZeroInDate = true @@ -1954,7 +1959,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.PrevLastInsertID = vars.StmtCtx.PrevLastInsertID } sc.PrevAffectedRows = 0 - if vars.StmtCtx.InUpdateStmt || vars.StmtCtx.InDeleteStmt || vars.StmtCtx.InInsertStmt { + if vars.StmtCtx.InUpdateStmt || vars.StmtCtx.InDeleteStmt || vars.StmtCtx.InInsertStmt || vars.StmtCtx.InSetSessionStatesStmt { sc.PrevAffectedRows = int64(vars.StmtCtx.AffectedRows()) } else if vars.StmtCtx.InSelectStmt { sc.PrevAffectedRows = -1 diff --git a/session/session.go b/session/session.go index c5c1ead4c65b4..d01c3e7e549d3 100644 --- a/session/session.go +++ b/session/session.go @@ -3543,15 +3543,16 @@ func (s *session) EncodeSessionStates(ctx context.Context, sctx sessionctx.Conte // DecodeSessionStates implements SessionStatesHandler.DecodeSessionStates interface. func (s *session) DecodeSessionStates(ctx context.Context, sctx sessionctx.Context, sessionStates *sessionstates.SessionStates) (err error) { - if err = s.sessionVars.DecodeSessionStates(ctx, sessionStates); err != nil { - return err - } - // Decode session variables. for name, val := range sessionStates.SystemVars { if err = variable.SetSessionSystemVar(s.sessionVars, name, val); err != nil { return err } } + + // Decode stmt ctx after session vars because setting session vars may override stmt ctx, such as warnings. + if err = s.sessionVars.DecodeSessionStates(ctx, sessionStates); err != nil { + return err + } return err } diff --git a/sessionctx/sessionstates/session_states.go b/sessionctx/sessionstates/session_states.go index baf876ff87b4f..10a2756dd04f4 100644 --- a/sessionctx/sessionstates/session_states.go +++ b/sessionctx/sessionstates/session_states.go @@ -18,6 +18,7 @@ import ( "time" ptypes "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" ) @@ -52,4 +53,7 @@ type SessionStates struct { FoundInBinding bool `json:"in-binding,omitempty"` SequenceLatestValues map[int64]int64 `json:"seq-values,omitempty"` MPPStoreLastFailTime map[string]time.Time `json:"store-fail-time,omitempty"` + LastAffectedRows int64 `json:"affected-rows,omitempty"` + LastInsertID uint64 `json:"last-insert-id,omitempty"` + Warnings []stmtctx.SQLWarn `json:"warnings,omitempty"` } diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 847f50f4e9a2b..29101af06f392 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -435,6 +435,125 @@ func TestSessionCtx(t *testing.T) { } } +func TestStatementCtx(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create table test.t1(id int auto_increment primary key, str char(1))") + + tests := []struct { + setFunc func(tk *testkit.TestKit) any + checkFunc func(tk *testkit.TestKit, param any) + }{ + { + // check LastAffectedRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("show warnings") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select row_count()").Check(testkit.Rows("0")) + tk.MustQuery("select row_count()").Check(testkit.Rows("-1")) + }, + }, + { + // check LastAffectedRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("select 1") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select row_count()").Check(testkit.Rows("-1")) + }, + }, + { + // check LastAffectedRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("insert into test.t1(str) value('a'), ('b'), ('c')") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select row_count()").Check(testkit.Rows("3")) + tk.MustQuery("select row_count()").Check(testkit.Rows("-1")) + }, + }, + { + // check LastInsertID + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_insert_id").Check(testkit.Rows("0")) + }, + }, + { + // check LastInsertID + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("insert into test.t1(str) value('d')") + rows := tk.MustQuery("select @@last_insert_id").Rows() + require.NotEqual(t, "0", rows[0][0].(string)) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_insert_id").Check(param.([][]any)) + }, + }, + { + // check Warning + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("select 1") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("show errors").Check(testkit.Rows()) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustQuery("show errors").Check(testkit.Rows()) + tk.MustQuery("select @@warning_count, @@error_count").Check(testkit.Rows("0 0")) + }, + }, + { + // check Warning + setFunc: func(tk *testkit.TestKit) any { + tk.MustGetErrCode("insert into test.t1(str) value('ef')", errno.ErrDataTooLong) + rows := tk.MustQuery("show warnings").Rows() + require.Equal(t, 1, len(rows)) + tk.MustQuery("show errors").Check(rows) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("show warnings").Check(param.([][]any)) + tk.MustQuery("show errors").Check(param.([][]any)) + tk.MustQuery("select @@warning_count, @@error_count").Check(testkit.Rows("1 1")) + }, + }, + { + // check Warning + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("set sql_mode=''") + tk.MustExec("insert into test.t1(str) value('ef'), ('ef')") + rows := tk.MustQuery("show warnings").Rows() + require.Equal(t, 2, len(rows)) + tk.MustQuery("show errors").Check(testkit.Rows()) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("show warnings").Check(param.([][]any)) + tk.MustQuery("show errors").Check(testkit.Rows()) + tk.MustQuery("select @@warning_count, @@error_count").Check(testkit.Rows("2 0")) + }, + }, + } + + for _, tt := range tests { + tk1 := testkit.NewTestKit(t, store) + var param any + if tt.setFunc != nil { + param = tt.setFunc(tk1) + } + tk2 := testkit.NewTestKit(t, store) + showSessionStatesAndSet(t, tk1, tk2) + tt.checkFunc(tk2, param) + } +} + func showSessionStatesAndSet(t *testing.T, tk1, tk2 *testkit.TestKit) { rows := tk1.MustQuery("show session_states").Rows() require.Len(t, rows, 1) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 70cd4bec5f898..4d623015492cc 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -15,6 +15,7 @@ package stmtctx import ( + "encoding/json" "math" "sort" "strconv" @@ -22,10 +23,12 @@ import ( "sync/atomic" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" @@ -60,6 +63,43 @@ type SQLWarn struct { Err error } +type jsonSQLWarn struct { + Level string `json:"level"` + SQLErr *terror.Error `json:"err,omitempty"` + Msg string `json:"msg,omitempty"` +} + +// MarshalJSON implements the Marshaler.MarshalJSON interface. +func (warn *SQLWarn) MarshalJSON() ([]byte, error) { + w := &jsonSQLWarn{ + Level: warn.Level, + } + e := errors.Cause(warn.Err) + switch x := e.(type) { + case *terror.Error: + // Omit outter errors because only the most inner error matters. + w.SQLErr = x + default: + w.Msg = e.Error() + } + return json.Marshal(w) +} + +// UnmarshalJSON implements the Unmarshaler.UnmarshalJSON interface. +func (warn *SQLWarn) UnmarshalJSON(data []byte) error { + var w jsonSQLWarn + if err := json.Unmarshal(data, &w); err != nil { + return err + } + warn.Level = w.Level + if w.SQLErr != nil { + warn.Err = w.SQLErr + } else { + warn.Err = errors.New(w.Msg) + } + return nil +} + // StatementContext contains variables for a statement. // It should be reset before executing a statement. type StatementContext struct { @@ -76,6 +116,7 @@ type StatementContext struct { InLoadDataStmt bool InExplainStmt bool InCreateOrAlterStmt bool + InSetSessionStatesStmt bool InPreparedPlanBuilding bool IgnoreTruncate bool IgnoreZeroInDate bool @@ -406,6 +447,13 @@ func (sc *StatementContext) AddAffectedRows(rows uint64) { sc.mu.affectedRows += rows } +// SetAffectedRows sets affected rows. +func (sc *StatementContext) SetAffectedRows(rows uint64) { + sc.mu.Lock() + sc.mu.affectedRows = rows + sc.mu.Unlock() +} + // AffectedRows gets affected rows. func (sc *StatementContext) AffectedRows() uint64 { sc.mu.Lock() @@ -558,6 +606,7 @@ func (sc *StatementContext) SetWarnings(warns []SQLWarn) { sc.mu.Lock() defer sc.mu.Unlock() sc.mu.warnings = warns + sc.mu.errorCount = 0 for _, w := range warns { if w.Level == WarnLevelError { sc.mu.errorCount++ diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index 7a4ec77a90660..b8f36dcb25055 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -16,12 +16,15 @@ package stmtctx_test import ( "context" + "encoding/json" "fmt" "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/execdetails" "github.com/stretchr/testify/require" @@ -143,3 +146,43 @@ func TestWeakConsistencyRead(t *testing.T) { execAndCheck("execute s", testkit.Rows("1 1 2"), kv.SI) tk.MustExec("rollback") } + +func TestMarshalSQLWarn(t *testing.T) { + warns := []stmtctx.SQLWarn{ + { + Level: stmtctx.WarnLevelError, + Err: errors.New("any error"), + }, + { + Level: stmtctx.WarnLevelError, + Err: errors.Trace(errors.New("any error")), + }, + { + Level: stmtctx.WarnLevelWarning, + Err: variable.ErrUnknownSystemVar.GenWithStackByArgs("unknown"), + }, + { + Level: stmtctx.WarnLevelWarning, + Err: errors.Trace(variable.ErrUnknownSystemVar.GenWithStackByArgs("unknown")), + }, + } + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + // First query can trigger loading global variables, which produces warnings. + tk.MustQuery("select 1") + tk.Session().GetSessionVars().StmtCtx.SetWarnings(warns) + rows := tk.MustQuery("show warnings").Rows() + require.Equal(t, len(warns), len(rows)) + + // The unmarshalled result doesn't need to be exactly the same with the original one. + // We only need that the results of `show warnings` are the same. + bytes, err := json.Marshal(warns) + require.NoError(t, err) + var newWarns []stmtctx.SQLWarn + err = json.Unmarshal(bytes, &newWarns) + require.NoError(t, err) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(newWarns) + tk.MustQuery("show warnings").Check(rows) +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index fe4f469e76134..12546cde3c0ad 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1867,6 +1867,11 @@ func (s *SessionVars) EncodeSessionStates(ctx context.Context, sessionStates *se sessionStates.MPPStoreLastFailTime = s.MPPStoreLastFailTime sessionStates.FoundInPlanCache = s.PrevFoundInPlanCache sessionStates.FoundInBinding = s.PrevFoundInBinding + + // Encode StatementContext. We encode it here to avoid circle dependency. + sessionStates.LastAffectedRows = s.StmtCtx.PrevAffectedRows + sessionStates.LastInsertID = s.StmtCtx.PrevLastInsertID + sessionStates.Warnings = s.StmtCtx.GetWarnings() return } @@ -1902,6 +1907,11 @@ func (s *SessionVars) DecodeSessionStates(ctx context.Context, sessionStates *se } s.FoundInPlanCache = sessionStates.FoundInPlanCache s.FoundInBinding = sessionStates.FoundInBinding + + // Decode StatementContext. + s.StmtCtx.SetAffectedRows(uint64(sessionStates.LastAffectedRows)) + s.StmtCtx.PrevLastInsertID = sessionStates.LastInsertID + s.StmtCtx.SetWarnings(sessionStates.Warnings) return } From ab27d4918a5a07d565ce3ef98761b6de6e90c9cc Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 27 Jun 2022 14:24:38 +0800 Subject: [PATCH 04/28] planner: fix the wrong cost formula of MPPExchanger on cost model ver2 (#35718) ref pingcap/tidb#35240 --- planner/core/plan_cost.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/planner/core/plan_cost.go b/planner/core/plan_cost.go index 5612933b9cf9b..73758b562536a 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -1209,8 +1209,12 @@ func (p *PhysicalExchangeReceiver) GetPlanCost(taskType property.TaskType, costF } p.planCost = childCost // accumulate net cost - // TODO: this formula is wrong since it doesn't consider tableRowSize, fix it later - p.planCost += getCardinality(p.children[0], costFlag) * p.ctx.GetSessionVars().GetNetworkFactor(nil) + if p.ctx.GetSessionVars().CostModelVersion == modelVer1 { + p.planCost += getCardinality(p.children[0], costFlag) * p.ctx.GetSessionVars().GetNetworkFactor(nil) + } else { // to avoid regression, only consider row-size on model ver2 + rowSize := getTblStats(p.children[0]).GetAvgRowSize(p.ctx, p.children[0].Schema().Columns, false, false) + p.planCost += getCardinality(p.children[0], costFlag) * rowSize * p.ctx.GetSessionVars().GetNetworkFactor(nil) + } p.planCostInit = true return p.planCost, nil } From d99b35822500590d95525f3f9c6b9597bab764cd Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 28 Jun 2022 12:08:39 +0800 Subject: [PATCH 05/28] *: only add default value for final aggregation to fix the aggregate push down (partition) union case (#35443) close pingcap/tidb#35295 --- executor/aggregate_test.go | 40 ++++++++++++++++++++++ executor/builder.go | 10 ++++-- expression/aggregation/descriptor.go | 2 -- planner/core/physical_plans.go | 2 +- planner/core/rule_aggregation_push_down.go | 10 ++++++ planner/core/rule_eliminate_projection.go | 4 +-- planner/core/task.go | 22 ++++++++++-- 7 files changed, 80 insertions(+), 10 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 764b2e624d7ac..f36e066505ad1 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1589,3 +1589,43 @@ func TestRandomPanicAggConsume(t *testing.T) { require.EqualError(t, err, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") } } + +func TestIssue35295(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t100") + // This bug only happens on partition prune mode = 'static' + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec(`CREATE TABLE t100 ( +ID bigint(20) unsigned NOT NULL AUTO_INCREMENT, +col1 int(10) NOT NULL DEFAULT '0' COMMENT 'test', +money bigint(20) NOT NULL COMMENT 'test', +logtime datetime NOT NULL COMMENT '记录时间', +PRIMARY KEY (ID,logtime) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin AUTO_INCREMENT=1 COMMENT='test' +PARTITION BY RANGE COLUMNS(logtime) ( +PARTITION p20220608 VALUES LESS THAN ("20220609"), +PARTITION p20220609 VALUES LESS THAN ("20220610"), +PARTITION p20220610 VALUES LESS THAN ("20220611"), +PARTITION p20220611 VALUES LESS THAN ("20220612"), +PARTITION p20220612 VALUES LESS THAN ("20220613"), +PARTITION p20220613 VALUES LESS THAN ("20220614"), +PARTITION p20220614 VALUES LESS THAN ("20220615"), +PARTITION p20220615 VALUES LESS THAN ("20220616"), +PARTITION p20220616 VALUES LESS THAN ("20220617"), +PARTITION p20220617 VALUES LESS THAN ("20220618"), +PARTITION p20220618 VALUES LESS THAN ("20220619"), +PARTITION p20220619 VALUES LESS THAN ("20220620"), +PARTITION p20220620 VALUES LESS THAN ("20220621"), +PARTITION p20220621 VALUES LESS THAN ("20220622"), +PARTITION p20220622 VALUES LESS THAN ("20220623"), +PARTITION p20220623 VALUES LESS THAN ("20220624"), +PARTITION p20220624 VALUES LESS THAN ("20220625") + );`) + tk.MustExec("insert into t100(col1,money,logtime) values (100,10,'2022-06-09 00:00:00');") + tk.MustExec("insert into t100(col1,money,logtime) values (100,10,'2022-06-10 00:00:00');") + tk.MustQuery("SELECT /*+STREAM_AGG()*/ col1,sum(money) FROM t100 WHERE logtime>='2022-06-09 00:00:00' AND col1=100 ;").Check(testkit.Rows("100 20")) + tk.MustQuery("SELECT /*+HASH_AGG()*/ col1,sum(money) FROM t100 WHERE logtime>='2022-06-09 00:00:00' AND col1=100 ;").Check(testkit.Rows("100 20")) +} diff --git a/executor/builder.go b/executor/builder.go index 73a4995696de0..db0acf8eb02aa 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1434,7 +1434,9 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor if len(v.GroupByItems) != 0 || aggregation.IsAllFirstRow(v.AggFuncs) { e.defaultVal = nil } else { - e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + if v.IsFinalAgg() { + e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + } } for _, aggDesc := range v.AggFuncs { if aggDesc.HasDistinct || len(aggDesc.OrderByItems) > 0 { @@ -1490,10 +1492,14 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu groupChecker: newVecGroupChecker(b.ctx, v.GroupByItems), aggFuncs: make([]aggfuncs.AggFunc, 0, len(v.AggFuncs)), } + if len(v.GroupByItems) != 0 || aggregation.IsAllFirstRow(v.AggFuncs) { e.defaultVal = nil } else { - e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + // Only do this for final agg, see issue #35295, #30923 + if v.IsFinalAgg() { + e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + } } for i, aggDesc := range v.AggFuncs { aggFunc := aggfuncs.Build(b.ctx, aggDesc, i) diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 115559022279c..debeaf8d6399e 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -123,8 +123,6 @@ func (a *AggFuncDesc) Split(ordinal []int) (partialAggDesc, finalAggDesc *AggFun partialAggDesc.Mode = Partial1Mode } else if a.Mode == FinalMode { partialAggDesc.Mode = Partial2Mode - } else { - panic("Error happened during AggFuncDesc.Split, the AggFunctionMode is not CompleteMode or FinalMode.") } finalAggDesc = &AggFuncDesc{ Mode: FinalMode, // We only support FinalMode now in final phase. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 9f5721049c166..c5c43df0a0a91 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1077,7 +1077,7 @@ type basePhysicalAgg struct { MppPartitionCols []*property.MPPPartitionColumn } -func (p *basePhysicalAgg) isFinalAgg() bool { +func (p *basePhysicalAgg) IsFinalAgg() bool { if len(p.AggFuncs) > 0 { if p.AggFuncs[0].Mode == aggregation.FinalMode || p.AggFuncs[0].Mode == aggregation.CompleteMode { return true diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 826320ffdee71..11508a6ef0688 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -438,6 +438,16 @@ func (a *aggregationPushDownSolver) tryAggPushDownForUnion(union *LogicalUnionAl if pushedAgg == nil { return nil } + + // Update the agg mode for the pushed down aggregation. + for _, aggFunc := range pushedAgg.AggFuncs { + if aggFunc.Mode == aggregation.CompleteMode { + aggFunc.Mode = aggregation.Partial1Mode + } else if aggFunc.Mode == aggregation.FinalMode { + aggFunc.Mode = aggregation.Partial2Mode + } + } + newChildren := make([]LogicalPlan, 0, len(union.Children())) for _, child := range union.Children() { newChild, err := a.pushAggCrossUnion(pushedAgg, union.Schema(), child) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 5516a242b486d..70a55f7e4e339 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -49,14 +49,14 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { // passing down the aggregation mode to TiFlash. if physicalAgg, ok := p.Children()[0].(*PhysicalHashAgg); ok { if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar { - if physicalAgg.isFinalAgg() { + if physicalAgg.IsFinalAgg() { return false } } } if physicalAgg, ok := p.Children()[0].(*PhysicalStreamAgg); ok { if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar { - if physicalAgg.isFinalAgg() { + if physicalAgg.IsFinalAgg() { return false } } diff --git a/planner/core/task.go b/planner/core/task.go index e5615e1c46f9b..2f6d853f6b382 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1337,7 +1337,15 @@ func BuildFinalModeAggregation( finalAggFunc.OrderByItems = byItems finalAggFunc.HasDistinct = aggFunc.HasDistinct - finalAggFunc.Mode = aggregation.CompleteMode + // In logical optimize phase, the Agg->PartitionUnion->TableReader may become + // Agg1->PartitionUnion->Agg2->TableReader, and the Agg2 is a partial aggregation. + // So in the push down here, we need to add a new if-condition check: + // If the original agg mode is partial already, the finalAggFunc's mode become Partial2. + if aggFunc.Mode == aggregation.CompleteMode { + finalAggFunc.Mode = aggregation.CompleteMode + } else if aggFunc.Mode == aggregation.Partial1Mode || aggFunc.Mode == aggregation.Partial2Mode { + finalAggFunc.Mode = aggregation.Partial2Mode + } } else { if aggFunc.Name == ast.AggFuncGroupConcat && len(aggFunc.OrderByItems) > 0 { // group_concat can only run in one phase if it has order by items but without distinct property @@ -1417,7 +1425,15 @@ func BuildFinalModeAggregation( } } - finalAggFunc.Mode = aggregation.FinalMode + // In logical optimize phase, the Agg->PartitionUnion->TableReader may become + // Agg1->PartitionUnion->Agg2->TableReader, and the Agg2 is a partial aggregation. + // So in the push down here, we need to add a new if-condition check: + // If the original agg mode is partial already, the finalAggFunc's mode become Partial2. + if aggFunc.Mode == aggregation.CompleteMode { + finalAggFunc.Mode = aggregation.FinalMode + } else if aggFunc.Mode == aggregation.Partial1Mode || aggFunc.Mode == aggregation.Partial2Mode { + finalAggFunc.Mode = aggregation.Partial2Mode + } } finalAggFunc.Args = args @@ -1483,7 +1499,7 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { } // no avgs // for final agg, always add project due to in-compatibility between TiDB and TiFlash - if len(p.schema.Columns) == len(newSchema.Columns) && !p.isFinalAgg() { + if len(p.schema.Columns) == len(newSchema.Columns) && !p.IsFinalAgg() { return nil } // add remaining columns to exprs From f8552be79f05f6739d98b408ddd83faaefa49d63 Mon Sep 17 00:00:00 2001 From: Baoshuo Ren Date: Tue, 28 Jun 2022 12:30:39 +0800 Subject: [PATCH 06/28] readme: remove adopters (pingcap/docs#8725) (#35124) ref pingcap/tidb#35180 --- README.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/README.md b/README.md index ca96703510e40..36669b3de6682 100644 --- a/README.md +++ b/README.md @@ -62,10 +62,6 @@ The [community repository](https://github.com/pingcap/community) hosts all infor Contributions are welcomed and greatly appreciated. All the contributors are welcomed to claim your reward by filing this [form](https://forms.pingcap.com/f/tidb-contribution-swag). See [Contribution to TiDB](https://pingcap.github.io/tidb-dev-guide/contribute-to-tidb/introduction.html) for details on typical contribution workflows. For more contributing information, click on the contributor icon above. -## Adopters - -View the current list of in-production TiDB adopters [here](https://docs.pingcap.com/tidb/stable/adopters). - ## Case studies - [English](https://pingcap.com/case-studies) From 82f0faf78a6df2d9feb852ca33d65def40771b49 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 28 Jun 2022 13:08:39 +0800 Subject: [PATCH 07/28] executor: fix left join on partition table generate invalid lock key (#35732) close pingcap/tidb#28073 --- executor/executor.go | 9 +++++++++ executor/union_scan_test.go | 24 ++++++++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/executor/executor.go b/executor/executor.go index 491f3b2b4e27a..9bb2cd5789d65 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1026,6 +1026,15 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { physTblID := tblID if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) + if physTblID == 0 { + // select * from t1 left join t2 on t1.c = t2.c for update + // The join right side might be added NULL in left join + // In that case, physTblID is 0, so skip adding the lock. + // + // Note, we can't distinguish whether it's the left join case, + // or a bug that TiKV return without correct physical ID column. + continue + } } e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 4979af65273bd..b585ba5f20c13 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -15,11 +15,13 @@ package executor_test import ( + "encoding/hex" "fmt" "testing" "time" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/benchdaily" @@ -456,6 +458,28 @@ func TestIssue28073(t *testing.T) { break } require.False(t, exist) + + // Another case, left join on partition table should not generate locks on physical ID = 0 + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int, c_str));") + tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int)) partition by hash (c_int) partitions 4;") + tk.MustExec("insert into t1 (`c_int`, `c_str`) values (1, 'upbeat solomon'), (5, 'sharp rubin');") + tk.MustExec("insert into t2 (`c_int`, `c_str`) values (1, 'clever haibt'), (4, 'kind margulis');") + tk.MustExec("begin pessimistic;") + tk.MustQuery("select * from t1 left join t2 on t1.c_int = t2.c_int for update;").Check(testkit.Rows( + "1 upbeat solomon 1 clever haibt", + "5 sharp rubin ", + )) + key, err := hex.DecodeString("7480000000000000005F728000000000000000") + require.NoError(t, err) + h := helper.NewHelper(store.(helper.Storage)) + resp, err := h.GetMvccByEncodedKey(key) + require.NoError(t, err) + require.Nil(t, resp.Info.Lock) + require.Len(t, resp.Info.Writes, 0) + require.Len(t, resp.Info.Values, 0) + + tk.MustExec("rollback;") } func TestIssue32422(t *testing.T) { From 1f40fc72a586d318bb192072366e4555968f5f07 Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Tue, 28 Jun 2022 14:02:39 +0800 Subject: [PATCH 08/28] expression: use cloned RetType at `evaluateExprWithNull` when it may be changed. (#35759) close pingcap/tidb#25813 --- expression/expression.go | 2 +- expression/expression_test.go | 24 ++++++++++++++++++++++++ planner/core/integration_test.go | 19 +++++++++++++++++++ 3 files changed, 44 insertions(+), 1 deletion(-) diff --git a/expression/expression.go b/expression/expression.go index ad434dc5e9b5d..9965ce0a7cecc 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -807,7 +807,7 @@ func evaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expressio for i, arg := range x.GetArgs() { args[i] = evaluateExprWithNull(ctx, schema, arg) } - return NewFunctionInternal(ctx, x.FuncName.L, x.RetType, args...) + return NewFunctionInternal(ctx, x.FuncName.L, x.RetType.Clone(), args...) case *Column: if !schema.Contains(x) { return x diff --git a/expression/expression_test.go b/expression/expression_test.go index 499f5c0bfea21..00a2c1bee360c 100644 --- a/expression/expression_test.go +++ b/expression/expression_test.go @@ -79,6 +79,30 @@ func TestEvaluateExprWithNullAndParameters(t *testing.T) { require.True(t, isScalarFunc) // the expression with parameters is not evaluated } +func TestEvaluateExprWithNullNoChangeRetType(t *testing.T) { + ctx := createContext(t) + tblInfo := newTestTableBuilder("").add("col_str", mysql.TypeString, 0).build() + schema := tableInfoToSchemaForTest(tblInfo) + + castStrAsJSON := BuildCastFunction(ctx, schema.Columns[0], types.NewFieldType(mysql.TypeJSON)) + jsonConstant := &Constant{Value: types.NewDatum("123"), RetType: types.NewFieldType(mysql.TypeJSON)} + + // initially has ParseToJSONFlag + flagInCast := castStrAsJSON.(*ScalarFunction).RetType.GetFlag() + require.True(t, mysql.HasParseToJSONFlag(flagInCast)) + + // cast's ParseToJSONFlag removed by `DisableParseJSONFlag4Expr` + eq, err := newFunctionForTest(ctx, ast.EQ, jsonConstant, castStrAsJSON) + require.NoError(t, err) + flagInCast = eq.(*ScalarFunction).GetArgs()[1].(*ScalarFunction).RetType.GetFlag() + require.False(t, mysql.HasParseToJSONFlag(flagInCast)) + + // after EvaluateExprWithNull, this flag should be still false + EvaluateExprWithNull(ctx, schema, eq) + flagInCast = eq.(*ScalarFunction).GetArgs()[1].(*ScalarFunction).RetType.GetFlag() + require.False(t, mysql.HasParseToJSONFlag(flagInCast)) +} + func TestConstant(t *testing.T) { ctx := createContext(t) sc := &stmtctx.StatementContext{TimeZone: time.Local} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index c0890093b0080..9fe9a7192cdcb 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -6717,3 +6717,22 @@ func TestIssue35083(t *testing.T) { "└─Projection 10000.00 cop[tikv] cast(test.t1.a, datetime BINARY)->Column#4", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) } + +func TestIssue25813(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a json);") + tk.MustExec("insert into t values('{\"id\": \"ish\"}');") + tk.MustQuery("select t2.a from t t1 left join t t2 on t1.a=t2.a where t2.a->'$.id'='ish';").Check(testkit.Rows("{\"id\": \"ish\"}")) + + tk.MustQuery("explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where t2.a->'$.id'='ish';").Check(testkit.Rows( + "Selection 8000.00 root eq(json_extract(test.t.a, \"$.id\"), cast(\"ish\", json BINARY))", + "└─HashJoin 10000.00 root left outer join, equal:[eq(test.t.a, test.t.a)]", + " ├─TableReader(Build) 8000.00 root data:Selection", + " │ └─Selection 8000.00 cop[tikv] not(isnull(cast(test.t.a, var_string(4294967295))))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) +} From dd7809e2b29d893f87f7a000ae590fd704594f20 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 28 Jun 2022 15:14:39 +0800 Subject: [PATCH 09/28] metrics: add cached table related metrics to grafana panel (#34718) close pingcap/tidb#34717 --- metrics/grafana/tidb.json | 585 ++++++++++++++++++++++++++++++++++---- 1 file changed, 523 insertions(+), 62 deletions(-) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index fcb3dfa5b55ca..30d2c788dc0e1 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -14,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "6.4.5" + "version": "7.5.11" }, { "type": "panel", @@ -51,12 +51,13 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": 28, - "iteration": 1617019008967, + "id": null, + "iteration": 1655990780337, "links": [], "panels": [ { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -72,13 +73,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB query durations by histogram buckets with different percents", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, "y": 1 }, + "hiddenSeries": false, "id": 80, "legend": { "alignAsTable": false, @@ -97,7 +104,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -185,7 +196,12 @@ "description": "MySQL commands processing numbers per second. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 6, @@ -193,6 +209,7 @@ "x": 12, "y": 1 }, + "hiddenSeries": false, "id": 42, "legend": { "alignAsTable": false, @@ -215,7 +232,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -306,7 +327,12 @@ "description": "TiDB statement statistics", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 6, @@ -314,6 +340,7 @@ "x": 0, "y": 7 }, + "hiddenSeries": false, "id": 21, "legend": { "alignAsTable": true, @@ -334,7 +361,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -410,7 +441,12 @@ "description": "TiDB command total statistics including both successful and failed ones", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 6, @@ -418,6 +454,7 @@ "x": 12, "y": 7 }, + "hiddenSeries": false, "id": 2, "legend": { "alignAsTable": true, @@ -440,7 +477,11 @@ "links": [], "maxPerRow": 1, "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -513,13 +554,19 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": null, "description": "TiDB failed query statistics by query type", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 6, "w": 24, "x": 0, "y": 13 }, + "hiddenSeries": false, "id": 137, "legend": { "alignAsTable": true, @@ -539,7 +586,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -604,13 +655,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB slow query statistics with slow query durations and coprocessor waiting/executing durations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, - "y": 25 + "y": 19 }, + "hiddenSeries": false, "id": 112, "legend": { "avg": false, @@ -625,7 +682,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -704,13 +765,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB connection idle durations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 12, - "y": 25 + "y": 19 }, + "hiddenSeries": false, "id": 218, "legend": { "avg": false, @@ -725,7 +792,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -831,13 +902,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 99.9 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, "y": 25 }, + "hiddenSeries": false, "id": 136, "legend": { "avg": false, @@ -852,7 +929,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -917,13 +998,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 99 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 12, "y": 25 }, + "hiddenSeries": false, "id": 134, "legend": { "avg": false, @@ -938,7 +1025,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -1003,13 +1094,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 95 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 0, "y": 31 }, + "hiddenSeries": false, "id": 132, "legend": { "avg": false, @@ -1024,7 +1121,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -1089,13 +1190,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB durations for different query types with 80 percent buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 6, "w": 12, "x": 12, "y": 31 }, + "hiddenSeries": false, "id": 130, "legend": { "avg": false, @@ -1110,7 +1217,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -1175,6 +1286,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -1751,6 +1863,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -3337,11 +3450,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 3 }, "id": 141, "panels": [ @@ -5583,11 +5697,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 4 }, "id": 142, "panels": [ @@ -5601,14 +5716,20 @@ "description": "The time cost of parsing SQL to AST", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 82 + "y": 5 }, + "hiddenSeries": false, "id": 156, "interval": "", "legend": { @@ -5630,7 +5751,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5702,14 +5827,20 @@ "description": "The time cost of building the query plan", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 82 + "y": 5 }, + "hiddenSeries": false, "id": 154, "interval": "", "legend": { @@ -5731,7 +5862,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5803,14 +5938,20 @@ "description": "The time cost of executing the SQL which does not include the time to get the results of the query .", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 90 + "y": 13 }, + "hiddenSeries": false, "id": 169, "interval": "", "legend": { @@ -5832,7 +5973,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5904,14 +6049,20 @@ "description": "TiDB executors using more cpu and memory resources", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, "x": 12, - "y": 90 + "y": 13 }, + "hiddenSeries": false, "id": 76, "legend": { "alignAsTable": true, @@ -5932,7 +6083,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -5954,7 +6109,115 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Expensive Executors OPS", + "title": "Expensive Executors OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 10, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "TiDB plan cache hit total", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 21 + }, + "hiddenSeries": false, + "id": 91, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": null, + "sortDesc": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_server_plan_cache_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 30 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Queries Using Plan Cache OPS", "tooltip": { "msResolution": false, "shared": true, @@ -5973,7 +6236,7 @@ { "format": "short", "label": null, - "logBase": 10, + "logBase": 2, "max": null, "min": "0", "show": true @@ -6002,15 +6265,21 @@ "description": "TiDB plan cache hit total", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 98 + "x": 12, + "y": 21 }, - "id": 91, + "hiddenSeries": false, + "id": 250, "legend": { "alignAsTable": true, "avg": true, @@ -6030,7 +6299,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -6097,17 +6370,23 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "TiDB plan cache miss total", + "description": "TiDB read table cache hit total", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 98 + "x": 0, + "y": 29 }, + "hiddenSeries": false, "id": 249, "legend": { "alignAsTable": true, @@ -6128,7 +6407,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -6138,10 +6421,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_plan_cache_miss_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(rate(tidb_server_read_from_tablecache_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", + "interval": "", "intervalFactor": 2, - "legendFormat": "{{type}}", + "legendFormat": "qps", "refId": "A", "step": 30 } @@ -6150,7 +6435,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Plan Cache Miss OPS", + "title": "Read From Table Cache OPS", "tooltip": { "msResolution": false, "shared": true, @@ -6167,6 +6452,7 @@ }, "yaxes": [ { + "$$hashKey": "object:469", "format": "short", "label": null, "logBase": 2, @@ -6175,6 +6461,7 @@ "show": true }, { + "$$hashKey": "object:470", "format": "short", "label": null, "logBase": 1, @@ -6195,11 +6482,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 5 }, "id": 143, "panels": [ @@ -6236,7 +6524,7 @@ "linewidth": 2, "links": [ { - "type": "dashboard" + "url": "/" } ], "nullPointMode": "null as zero", @@ -7007,11 +7295,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 6 }, "id": 144, "panels": [ @@ -7621,11 +7910,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 7 }, "id": 145, "panels": [ @@ -8105,11 +8395,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 9 + "y": 8 }, "id": 147, "panels": [ @@ -8824,8 +9115,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "It indicates if a request of PD client is forwarded by the PD follower", "decimals": null, + "description": "It indicates if a request of PD client is forwarded by the PD follower", "fill": 1, "gridPos": { "h": 7, @@ -9304,11 +9595,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 10 + "y": 9 }, "id": 148, "panels": [ @@ -9321,14 +9613,20 @@ "description": "TiDB loading schema time durations by instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 47 + "y": 10 }, + "hiddenSeries": false, "id": 27, "legend": { "alignAsTable": true, @@ -9345,7 +9643,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -9412,18 +9714,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB loading schema times including both failed and successful ones", + "description": "TiDB loading schema time durations by instance", "editable": true, "error": false, - "fill": 0, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 47 + "y": 10 }, - "id": 28, + "hiddenSeries": false, + "id": 251, "legend": { "alignAsTable": true, "avg": false, @@ -9439,26 +9747,25 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*failed/", - "bars": true - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_domain_load_schema_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,type)", + "expr": "histogram_quantile(0.99, sum(rate(tidb_domain_load_schema_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", - "metric": "tidb_domain_load_schema_duration_count", + "legendFormat": "{{instance}}", + "metric": "", "refId": "A", "step": 10 } @@ -9467,7 +9774,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Load Schema OPS", + "title": "Load Schema Duration", "tooltip": { "msResolution": false, "shared": true, @@ -9484,11 +9791,11 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 10, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -9515,14 +9822,20 @@ "description": "TiDB schema lease error counts", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 54 + "y": 17 }, + "hiddenSeries": false, "id": 29, "legend": { "alignAsTable": true, @@ -9541,7 +9854,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -9608,18 +9925,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB load privilege counts", + "description": "TiDB loading schema times including both failed and successful ones", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 54 + "y": 17 }, - "id": 157, + "hiddenSeries": false, + "id": 28, "legend": { "alignAsTable": true, "avg": false, @@ -9635,7 +9958,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -9650,7 +9977,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_domain_load_privilege_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,type)", + "expr": "sum(rate(tidb_domain_load_schema_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance,type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -9663,7 +9990,118 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Load Privilege OPS", + "title": "Load Schema OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 10, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB loading table cache time durations by instance", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 24 + }, + "hiddenSeries": false, + "id": 157, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*failed/", + "bars": true + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_load_table_cache_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "metric": "tidb_domain_load_schema_duration_count", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Load Data From Cached Table Duration", "tooltip": { "msResolution": false, "shared": true, @@ -9680,6 +10118,7 @@ }, "yaxes": [ { + "$$hashKey": "object:541", "format": "short", "label": null, "logBase": 10, @@ -9688,6 +10127,7 @@ "show": true }, { + "$$hashKey": "object:542", "format": "short", "label": null, "logBase": 1, @@ -9708,11 +10148,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 11 + "y": 10 }, "id": 149, "panels": [ @@ -10635,11 +11076,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 12 + "y": 11 }, "id": 150, "panels": [ @@ -11989,11 +12431,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 13 + "y": 12 }, "id": 161, "panels": [ @@ -12183,11 +12626,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 14 + "y": 13 }, "id": 151, "panels": [ @@ -12556,11 +13000,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 15 + "y": 14 }, "id": 152, "panels": [ @@ -13374,11 +13819,12 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 16 + "y": 15 }, "id": 178, "panels": [ @@ -13782,6 +14228,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -14443,7 +14890,7 @@ } ], "refresh": "30s", - "schemaVersion": 18, + "schemaVersion": 27, "style": "dark", "tags": [], "templating": { @@ -14452,15 +14899,22 @@ "allValue": null, "current": {}, "datasource": "${DS_TEST-CLUSTER}", + "definition": "", + "description": null, + "error": null, "hide": 2, "includeAll": false, "label": "K8s-cluster", "multi": false, "name": "k8s_cluster", "options": [], - "query": "label_values(pd_cluster_status, k8s_cluster)", + "query": { + "query": "label_values(pd_cluster_status, k8s_cluster)", + "refId": "test-cluster-k8s_cluster-Variable-Query" + }, "refresh": 2, "regex": "", + "skipUrlSync": false, "sort": 1, "tagValuesQuery": "", "tags": [], @@ -14472,15 +14926,22 @@ "allValue": null, "current": {}, "datasource": "${DS_TEST-CLUSTER}", + "definition": "", + "description": null, + "error": null, "hide": 2, "includeAll": false, "label": "tidb_cluster", "multi": false, "name": "tidb_cluster", "options": [], - "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "query": { + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "refId": "test-cluster-tidb_cluster-Variable-Query" + }, "refresh": 2, "regex": "", + "skipUrlSync": false, "sort": 1, "tagValuesQuery": "", "tags": [], From 8097685436fd3df473dd7f6721a572742c5f1cd4 Mon Sep 17 00:00:00 2001 From: tangenta Date: Tue, 28 Jun 2022 16:06:39 +0800 Subject: [PATCH 10/28] helper: request another PD if one of them is unavailable (#35750) close pingcap/tidb#35708 --- store/helper/helper.go | 32 ++++++++++++++++++-------------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/store/helper/helper.go b/store/helper/helper.go index e3912ac9aff26..948a84c84635d 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -840,42 +840,46 @@ func (h *Helper) requestPD(apiName, method, uri string, body io.Reader, res inte if len(pdHosts) == 0 { return errors.New("pd unavailable") } - logutil.BgLogger().Debug("RequestPD URL", zap.String("url", util.InternalHTTPSchema()+"://"+pdHosts[0]+uri)) - req := new(http.Request) for _, host := range pdHosts { - req, err = http.NewRequest(method, util.InternalHTTPSchema()+"://"+host+uri, body) - if err != nil { - // Try to request from another PD node when some nodes may down. - if strings.Contains(err.Error(), "connection refused") { - continue - } - return errors.Trace(err) + err = requestPDForOneHost(host, apiName, method, uri, body, res) + if err == nil { + break } + // Try to request from another PD node when some nodes may down. } + return err +} + +func requestPDForOneHost(host, apiName, method, uri string, body io.Reader, res interface{}) error { + urlVar := fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), host, uri) + logutil.BgLogger().Debug("RequestPD URL", zap.String("url", urlVar)) + req, err := http.NewRequest(method, urlVar, body) if err != nil { - return err + logutil.BgLogger().Warn("requestPDForOneHost new request failed", + zap.String("url", urlVar), zap.Error(err)) + return errors.Trace(err) } start := time.Now() resp, err := util.InternalHTTPClient().Do(req) if err != nil { metrics.PDAPIRequestCounter.WithLabelValues(apiName, "network error").Inc() + logutil.BgLogger().Warn("requestPDForOneHost do request failed", + zap.String("url", urlVar), zap.Error(err)) return errors.Trace(err) } metrics.PDAPIExecutionHistogram.WithLabelValues(apiName).Observe(time.Since(start).Seconds()) metrics.PDAPIRequestCounter.WithLabelValues(apiName, resp.Status).Inc() - defer func() { err = resp.Body.Close() if err != nil { - logutil.BgLogger().Error("close body failed", zap.Error(err)) + logutil.BgLogger().Warn("requestPDForOneHost close body failed", + zap.String("url", urlVar), zap.Error(err)) } }() - err = json.NewDecoder(resp.Body).Decode(res) if err != nil { return errors.Trace(err) } - return nil } From 27e7bbdd4e04d01869426a023ef1efaf85497c7e Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 28 Jun 2022 02:40:40 -0600 Subject: [PATCH 11/28] sessionctx/variable: add tests to ensure skipInit can be removed (#35703) ref pingcap/tidb#35051 --- sessionctx/variable/sysvar_test.go | 86 ++++++++++++++++++++++++++++++ 1 file changed, 86 insertions(+) diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index fb821c9f5cf78..68d68e42b159c 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -674,6 +674,92 @@ func TestSettersandGetters(t *testing.T) { } } +// TestSkipInitIsUsed ensures that no new variables are added with skipInit: true. +// This feature is deprecated, and if you need to run code to differentiate between init and "SET" (rare), +// you can instead check if s.StmtCtx.StmtType == "Set". +// The reason it is deprecated is that the behavior is typically wrong: +// it means session settings won't inherit from global and don't apply until you first set +// them in each session. This is a very weird behavior. +// See: https://github.com/pingcap/tidb/issues/35051 +func TestSkipInitIsUsed(t *testing.T) { + for _, sv := range GetSysVars() { + if sv.skipInit { + // Many of these variables might allow skipInit to be removed, + // they need to be checked first. The purpose of this test is to make + // sure we don't introduce any new variables with skipInit, which seems + // to be a problem. + switch sv.Name { + case Timestamp, + WarningCount, + ErrorCount, + LastInsertID, + Identity, + TiDBTxnScope, + TiDBSnapshot, + TiDBOptDistinctAggPushDown, + TiDBOptWriteRowID, + TiDBChecksumTableConcurrency, + TiDBBatchInsert, + TiDBBatchDelete, + TiDBBatchCommit, + TiDBCurrentTS, + TiDBLastTxnInfo, + TiDBLastQueryInfo, + TiDBEnableChunkRPC, + TxnIsolationOneShot, + TiDBOptimizerSelectivityLevel, + TiDBOptimizerEnableOuterJoinReorder, + TiDBLogFileMaxDays, + TiDBConfig, + TiDBDDLReorgPriority, + TiDBSlowQueryFile, + TiDBWaitSplitRegionFinish, + TiDBWaitSplitRegionTimeout, + TiDBLowResolutionTSO, + TiDBAllowRemoveAutoInc, + TiDBMetricSchemaStep, + TiDBMetricSchemaRangeDuration, + TiDBFoundInPlanCache, + TiDBFoundInBinding, + RandSeed1, + RandSeed2, + TiDBLastDDLInfo, + TiDBGeneralLog, + TiDBSlowLogThreshold, + TiDBRecordPlanInSlowLog, + TiDBEnableSlowLog, + TiDBCheckMb4ValueInUTF8, + TiDBPProfSQLCPU, + TiDBDDLSlowOprThreshold, + TiDBForcePriority, + TiDBMemoryUsageAlarmRatio, + TiDBEnableCollectExecutionInfo, + TiDBPersistAnalyzeOptions, + TiDBEnableColumnTracking, + TiDBStatsLoadPseudoTimeout, + SQLLogBin, + ForeignKeyChecks, + CollationDatabase, + CharacterSetClient, + CharacterSetResults, + CollationConnection, + CharsetDatabase, + GroupConcatMaxLen, + CharacterSetConnection, + CharacterSetServer, + TiDBBuildStatsConcurrency, + TiDBOptTiFlashConcurrencyFactor, + TiDBOptSeekFactor, + TiDBOptJoinReorderThreshold, + TiDBStatsLoadSyncWait, + CharacterSetFilesystem: + continue + } + require.Equal(t, false, sv.skipInit, fmt.Sprintf("skipInit should not be set on new system variables. variable %s is in violation", sv.Name)) + } + } +} + func TestSecureAuth(t *testing.T) { sv := GetSysVar(SecureAuth) vars := NewSessionVars() From 0d9e02b638ac3dd3775419197f6574e729f4846e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Tue, 28 Jun 2022 17:14:39 +0800 Subject: [PATCH 12/28] *: Add `testfork.RunTest` to run multiple tests in one function (#35746) close pingcap/tidb#35747 --- ddl/BUILD.bazel | 1 + sessiontxn/BUILD.bazel | 1 + sessiontxn/txn_context_test.go | 115 +++++++++--------- testkit/stepped.go | 6 +- testkit/testfork/BUILD.bazel | 18 +++ testkit/testfork/fork.go | 135 +++++++++++++++++++++ testkit/testfork/fork_test.go | 50 ++++++++ tests/realtikvtest/sessiontest/BUILD.bazel | 1 + 8 files changed, 264 insertions(+), 63 deletions(-) create mode 100644 testkit/testfork/BUILD.bazel create mode 100644 testkit/testfork/fork.go create mode 100644 testkit/testfork/fork_test.go diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 324d39bc469ec..807f65715202b 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -143,6 +143,7 @@ go_test( "integration_test.go", "main_test.go", "modify_column_test.go", + "multi_schema_change_test.go", "options_test.go", "partition_test.go", "placement_policy_ddl_test.go", diff --git a/sessiontxn/BUILD.bazel b/sessiontxn/BUILD.bazel index 922f80480ac34..bdb84de22657f 100644 --- a/sessiontxn/BUILD.bazel +++ b/sessiontxn/BUILD.bazel @@ -35,6 +35,7 @@ go_test( "//sessionctx", "//sessiontxn/staleread", "//testkit", + "//testkit/testfork", "//testkit/testsetup", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index e247a14c86f2b..9fec1844606a3 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testfork" "github.com/pingcap/tidb/testkit/testsetup" "github.com/stretchr/testify/require" "go.uber.org/goleak" @@ -725,65 +726,59 @@ func TestStillWriteConflictAfterRetry(t *testing.T) { "update t1 set v=v+1 where id in (1, 2, 3) and v>0", } - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - for _, isolation := range []string{ast.RepeatableRead, ast.ReadCommitted} { - for _, query := range queries { - for _, autocommit := range []bool{true, false} { - t.Run(fmt.Sprintf("%s,%s,autocommit=%v", isolation, query, autocommit), func(t *testing.T) { - tk.MustExec("truncate table t1") - tk.MustExec("insert into t1 values(1, 10)") - tk2 := testkit.NewSteppedTestKit(t, store) - defer tk2.MustExec("rollback") - - tk2.MustExec("use test") - tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") - tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", isolation)) - if autocommit { - tk2.MustExec("set autocommit=1") - tk2.MustExec("begin") - } else { - tk2.MustExec("set autocommit=0") - } - - tk2.SetBreakPoints( - sessiontxn.BreakPointBeforeExecutorFirstRun, - sessiontxn.BreakPointOnStmtRetryAfterLockError, - ) - - var isSelect, isUpdate bool - switch { - case strings.HasPrefix(query, "select"): - isSelect = true - tk2.SteppedMustQuery(query) - case strings.HasPrefix(query, "update"): - isUpdate = true - tk2.SteppedMustExec(query) - default: - require.FailNowf(t, "invalid query: ", query) - } - - // Pause the session before the executor first run and then update the record in another session - tk2.ExpectStopOnBreakPoint(sessiontxn.BreakPointBeforeExecutorFirstRun) - tk.MustExec("update t1 set v=v+1") - - // Session continues, it should get a lock error and retry, we pause the session before the executor's next run - // and then update the record in another session again. - tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) - tk.MustExec("update t1 set v=v+1") - - // Because the record is updated by another session again, when this session continues, it will get a lock error again. - tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) - tk2.Continue().ExpectIdle() - switch { - case isSelect: - tk2.GetQueryResult().Check(testkit.Rows("1 12")) - case isUpdate: - tk2.MustExec("commit") - tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) - } - }) - } + testfork.RunTest(t, func(t *testfork.T) { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("truncate table t1") + tk.MustExec("insert into t1 values(1, 10)") + tk2 := testkit.NewSteppedTestKit(t, store) + defer tk2.MustExec("rollback") + + tk2.MustExec("use test") + tk2.MustExec("set @@tidb_txn_mode = 'pessimistic'") + tk2.MustExec(fmt.Sprintf("set tx_isolation = '%s'", testfork.PickEnum(t, ast.RepeatableRead, ast.ReadCommitted))) + autocommit := testfork.PickEnum(t, 0, 1) + tk2.MustExec(fmt.Sprintf("set autocommit=%d", autocommit)) + if autocommit == 1 { + tk2.MustExec("begin") } - } + + tk2.SetBreakPoints( + sessiontxn.BreakPointBeforeExecutorFirstRun, + sessiontxn.BreakPointOnStmtRetryAfterLockError, + ) + + var isSelect, isUpdate bool + query := testfork.Pick(t, queries) + switch { + case strings.HasPrefix(query, "select"): + isSelect = true + tk2.SteppedMustQuery(query) + case strings.HasPrefix(query, "update"): + isUpdate = true + tk2.SteppedMustExec(query) + default: + require.FailNowf(t, "invalid query: ", query) + } + + // Pause the session before the executor first run and then update the record in another session + tk2.ExpectStopOnBreakPoint(sessiontxn.BreakPointBeforeExecutorFirstRun) + tk.MustExec("update t1 set v=v+1") + + // Session continues, it should get a lock error and retry, we pause the session before the executor's next run + // and then update the record in another session again. + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) + tk.MustExec("update t1 set v=v+1") + + // Because the record is updated by another session again, when this session continues, it will get a lock error again. + tk2.Continue().ExpectStopOnBreakPoint(sessiontxn.BreakPointOnStmtRetryAfterLockError) + tk2.Continue().ExpectIdle() + switch { + case isSelect: + tk2.GetQueryResult().Check(testkit.Rows("1 12")) + case isUpdate: + tk2.MustExec("commit") + tk2.MustQuery("select * from t1").Check(testkit.Rows("1 13")) + } + }) } diff --git a/testkit/stepped.go b/testkit/stepped.go index d63a2a1efc7b8..fc6e434ef85b9 100644 --- a/testkit/stepped.go +++ b/testkit/stepped.go @@ -80,7 +80,7 @@ func (ch steppedTestKitMsgChan) recvMsgWithCheck(tp steppedTestkitMsgType) (*ste } type steppedTestKitCommandContext struct { - t *testing.T + t testing.TB tk *TestKit notifyBreakPointAndWait func(string) } @@ -89,7 +89,7 @@ type steppedTestKitCommand func(ctx *steppedTestKitCommandContext) any // SteppedTestKit is the testkit that can run stepped command type SteppedTestKit struct { - t *testing.T + t testing.TB tk *TestKit // ch1 is used to send msg from foreground to background @@ -105,7 +105,7 @@ type SteppedTestKit struct { } // NewSteppedTestKit creates a new SteppedTestKit -func NewSteppedTestKit(t *testing.T, store kv.Storage) *SteppedTestKit { +func NewSteppedTestKit(t testing.TB, store kv.Storage) *SteppedTestKit { tk := &SteppedTestKit{ t: t, tk: NewTestKit(t, store), diff --git a/testkit/testfork/BUILD.bazel b/testkit/testfork/BUILD.bazel new file mode 100644 index 0000000000000..743bd70da5b0d --- /dev/null +++ b/testkit/testfork/BUILD.bazel @@ -0,0 +1,18 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "testfork", + srcs = ["fork.go"], + importpath = "github.com/pingcap/tidb/testkit/testfork", + visibility = ["//visibility:public"], + deps = [ + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) + +go_test( + name = "testfork_test", + srcs = ["fork_test.go"], + embed = [":testfork"], +) diff --git a/testkit/testfork/fork.go b/testkit/testfork/fork.go new file mode 100644 index 0000000000000..aa70039b2f042 --- /dev/null +++ b/testkit/testfork/fork.go @@ -0,0 +1,135 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testfork + +import ( + "fmt" + "os" + "strings" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +type pickStack struct { + stack [][]any + pos int + valid bool +} + +func newPickStack() *pickStack { + return &pickStack{ + valid: true, + } +} + +func (s *pickStack) NextStack() { + for len(s.stack) > 0 { + lastIndex := len(s.stack) - 1 + s.stack[lastIndex] = s.stack[lastIndex][1:] + if len(s.stack[lastIndex]) > 0 { + break + } + s.stack = s.stack[:lastIndex] + } + + s.pos = 0 + s.valid = len(s.stack) > 0 +} + +func (s *pickStack) PickValue(values []any) (any, error) { + if len(values) == 0 { + return nil, errors.New("values should not be empty") + } + + stackLen := len(s.stack) + if s.pos > stackLen { + return nil, errors.Newf("illegal state %d > %d", s.pos, stackLen) + } + + defer func() { + s.pos++ + }() + + if s.pos == stackLen { + s.stack = append(s.stack, values) + } + return s.stack[s.pos][0], nil +} + +func (s *pickStack) Values() []any { + values := make([]any, 0) + for _, v := range s.stack { + values = append(values, v[0]) + } + return values +} + +func (s *pickStack) ValuesText() string { + values := s.Values() + strValues := make([]string, len(values)) + for i, value := range values { + switch v := value.(type) { + case string: + strValues[i] = fmt.Sprintf(`"%s"`, v) + default: + strValues[i] = fmt.Sprintf("%v", v) + } + } + return "[" + strings.Join(strValues, " ") + "]" +} + +func (s *pickStack) Valid() bool { + return s.valid +} + +// T is used by for test +type T struct { + *testing.T + stack *pickStack +} + +// RunTest runs the test function `f` multiple times util all the values in `Pick` are tested. +func RunTest(t *testing.T, f func(t *T)) { + idx := 0 + for stack := newPickStack(); stack.Valid(); stack.NextStack() { + success := t.Run("", func(t *testing.T) { + f(&T{T: t, stack: stack}) + }) + + if !success { + _, err := fmt.Fprintf(os.Stderr, "SubTest #%v failed, failed values: %s\n", idx, stack.ValuesText()) + require.NoError(t, err) + } + idx++ + } +} + +// Pick returns a value from the values list +func Pick[E any](t *T, values []E) E { + slice := make([]any, len(values)) + for i, item := range values { + slice[i] = item + } + value, err := t.stack.PickValue(slice) + require.NoError(t, err) + return value.(E) +} + +// PickEnum returns a value from the value enums +func PickEnum[E any](t *T, item E, other ...E) E { + return Pick(t, append([]E{item}, other...)) +} diff --git a/testkit/testfork/fork_test.go b/testkit/testfork/fork_test.go new file mode 100644 index 0000000000000..64ed11b7a82e5 --- /dev/null +++ b/testkit/testfork/fork_test.go @@ -0,0 +1,50 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testfork + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestForkSubTest(t *testing.T) { + var values [][]any + RunTest(t, func(t *T) { + x := Pick(t, []int{1, 2, 3}) + y := PickEnum(t, "a", "b") + var z any + if x == 2 { + z = PickEnum(t, 10, 11) + } else { + z = Pick(t, []string{"g", "h"}) + } + values = append(values, []any{x, y, z}) + }) + require.Equal(t, [][]any{ + {1, "a", "g"}, + {1, "a", "h"}, + {1, "b", "g"}, + {1, "b", "h"}, + {2, "a", 10}, + {2, "a", 11}, + {2, "b", 10}, + {2, "b", 11}, + {3, "a", "g"}, + {3, "a", "h"}, + {3, "b", "g"}, + {3, "b", "h"}, + }, values) +} diff --git a/tests/realtikvtest/sessiontest/BUILD.bazel b/tests/realtikvtest/sessiontest/BUILD.bazel index 95c8ae77a9646..b0e5afb4dc0fd 100644 --- a/tests/realtikvtest/sessiontest/BUILD.bazel +++ b/tests/realtikvtest/sessiontest/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//kv", "//meta/autoid", "//parser", + "//parser/ast", "//parser/auth", "//parser/format", "//parser/model", From 78b0ae579c669cc1c57b4ba041c297d0cfa94e4a Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 28 Jun 2022 17:34:39 +0800 Subject: [PATCH 13/28] planner: fix TRACE PLAN TARGET = 'estimation' panic when meeting partition table (#35743) close pingcap/tidb#35117 --- planner/core/optimizer.go | 16 +++++++++++----- statistics/trace_test.go | 25 +++++++++++++++++++++++++ 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index e868160af9800..fde76b3a41eec 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -326,14 +326,20 @@ func refineCETrace(sctx sessionctx.Context) { return i.RowCount < j.RowCount }) traceRecords := stmtCtx.OptimizerCETrace - is := sctx.GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema) for _, rec := range traceRecords { tbl, ok := is.TableByID(rec.TableID) - if !ok { - logutil.BgLogger().Warn("[OptimizerTrace] Failed to find table in infoschema", - zap.Int64("table id", rec.TableID)) + if ok { + rec.TableName = tbl.Meta().Name.O + continue + } + tbl, _, _ = is.FindTableByPartitionID(rec.TableID) + if tbl != nil { + rec.TableName = tbl.Meta().Name.O + continue } - rec.TableName = tbl.Meta().Name.O + logutil.BgLogger().Warn("[OptimizerTrace] Failed to find table in infoschema", + zap.Int64("table id", rec.TableID)) } } diff --git a/statistics/trace_test.go b/statistics/trace_test.go index 03d3c71e7f8ca..144dbff27d530 100644 --- a/statistics/trace_test.go +++ b/statistics/trace_test.go @@ -100,3 +100,28 @@ func TestTraceCE(t *testing.T) { require.ElementsMatch(t, resultJSON, out[i].Trace) } } + +func TestTraceCEPartitionTable(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, d varchar(10), index idx(a, b)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN MAXVALUE);") + tk.MustExec(`insert into t values(1, 1, "aaa"), + (1, 1, "bbb"), + (1, 2, "ccc"), + (1, 2, "ddd"), + (2, 2, "aaa"), + (2, 3, "bbb")`) + tk.MustExec("analyze table t") + result := tk.MustQuery("trace plan target='estimation' select * from t where a >=1") + require.Len(t, result.Rows(), 1) + resultStr := result.Rows()[0][0].(string) + var resultJSON []*tracing.CETraceRecord + err := json.Unmarshal([]byte(resultStr), &resultJSON) + require.NoError(t, err) + for _, r := range resultJSON { + require.Equal(t, "t", r.TableName) + } +} From ed5e63a7a0a275e313cd78ab1777e46803c2de53 Mon Sep 17 00:00:00 2001 From: Meng Xin Date: Tue, 28 Jun 2022 18:26:39 +0800 Subject: [PATCH 14/28] types: fix incompatible implementation of jsonpath extraction (#35320) close pingcap/tidb#30352 --- types/json/binary_functions.go | 4 ++++ types/json/binary_test.go | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 88770ae78c89f..926ba52b8cca0 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -221,6 +221,10 @@ func (bj BinaryJSON) Extract(pathExprList []PathExpression) (ret BinaryJSON, fou // even if len(pathExprList) equals to 1. found = true ret = buf[0] + // Fix https://github.com/pingcap/tidb/issues/30352 + if pathExprList[0].ContainsAnyAsterisk() { + ret = buildBinaryArray(buf) + } } else { found = true ret = buildBinaryArray(buf) diff --git a/types/json/binary_test.go b/types/json/binary_test.go index e276d81def524..4142c9cdf150f 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -62,12 +62,12 @@ func TestBinaryJSONExtract(t *testing.T) { {bj1, []string{"$.*[0]"}, mustParseBinaryFromString(t, `["world", 1, true, "d"]`), true, nil}, {bj1, []string{`$.a[*]."aa"`}, mustParseBinaryFromString(t, `["bb", "cc"]`), true, nil}, {bj1, []string{`$."\"hello\""`}, mustParseBinaryFromString(t, `"world"`), true, nil}, - {bj1, []string{`$**[1]`}, mustParseBinaryFromString(t, `"2"`), true, nil}, + {bj1, []string{`$**[1]`}, mustParseBinaryFromString(t, `["2"]`), true, nil}, {bj3, []string{`$.properties.$type`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, {bj4, []string{`$.properties.$type$type`}, mustParseBinaryFromString(t, `{"$a$a" : "TiDB"}`), true, nil}, {bj4, []string{`$.properties.$type$type.$a$a`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, {bj5, []string{`$.properties.$type.$a.$b`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, - {bj5, []string{`$.properties.$type.$a.*[0]`}, mustParseBinaryFromString(t, `"TiDB"`), true, nil}, + {bj5, []string{`$.properties.$type.$a.*[0]`}, mustParseBinaryFromString(t, `["TiDB"]`), true, nil}, // test extract with multi path expressions. {bj1, []string{"$.a", "$[5]"}, mustParseBinaryFromString(t, `[[1, "2", {"aa": "bb"}, 4.0, {"aa": "cc"}]]`), true, nil}, From 0d6934466e0f4a85ad5a2620329f97f37496b99e Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 29 Jun 2022 10:32:40 +0800 Subject: [PATCH 15/28] ddl: DROP TABLE/VIEW/SEQUENCE now use XXXStmt as parameter (#35741) ref pingcap/tidb#35665 --- ddl/ddl.go | 6 +- ddl/ddl_api.go | 237 +++++++++++++++++++++++++++++++----------------- executor/ddl.go | 108 +--------------------- 3 files changed, 162 insertions(+), 189 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 0ea01e1222eb5..c86b8f60e7459 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -106,9 +106,9 @@ type DDL interface { DropSchema(ctx sessionctx.Context, stmt *ast.DropDatabaseStmt) error CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error CreateView(ctx sessionctx.Context, stmt *ast.CreateViewStmt) error - DropTable(ctx sessionctx.Context, tableIdent ast.Ident) (err error) + DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) RecoverTable(ctx sessionctx.Context, recoverInfo *RecoverInfo) (err error) - DropView(ctx sessionctx.Context, tableIdent ast.Ident) (err error) + DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) CreateIndex(ctx sessionctx.Context, tableIdent ast.Ident, keyType ast.IndexKeyType, indexName model.CIStr, columnNames []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error DropIndex(ctx sessionctx.Context, tableIdent ast.Ident, indexName model.CIStr, ifExists bool) error @@ -122,7 +122,7 @@ type DDL interface { UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error RepairTable(ctx sessionctx.Context, table *ast.TableName, createStmt *ast.CreateTableStmt) error CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error - DropSequence(ctx sessionctx.Context, tableIdent ast.Ident, ifExists bool) (err error) + DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceStmt) (err error) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error CreatePlacementPolicy(ctx sessionctx.Context, stmt *ast.CreatePlacementPolicyStmt) error DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacementPolicyStmt) error diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 01f57c3ce1d18..23a0de2c32eb6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5326,71 +5326,172 @@ func (d *ddl) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt return errors.Trace(err) } -// DropTable will proceed even if some table in the list does not exists. -func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) { - schema, tb, err := d.getSchemaAndTableByIdent(ctx, ti) - if err != nil { - return errors.Trace(err) - } +// If one drop those tables by mistake, it's difficult to recover. +// In the worst case, the whole TiDB cluster fails to bootstrap, so we prevent user from dropping them. +var systemTables = map[string]struct{}{ + "tidb": {}, + "gc_delete_range": {}, + "gc_delete_range_done": {}, +} - if tb.Meta().IsView() { - return infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name) - } - if tb.Meta().IsSequence() { - return infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name) +func isSystemTable(schema, table string) bool { + if schema != "mysql" { + return false } - if tb.Meta().TableCacheStatusType != model.TableCacheStatusDisable { - return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Table") + if _, ok := systemTables[table]; ok { + return true } + return false +} - job := &model.Job{ - SchemaID: schema.ID, - TableID: tb.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: schema.State, - TableName: tb.Meta().Name.L, - Type: model.ActionDropTable, - BinlogInfo: &model.HistoryInfo{}, - } +type objectType int + +const ( + tableObject objectType = iota + viewObject + sequenceObject +) + +// dropTableObject provides common logic to DROP TABLE/VIEW/SEQUENCE. +func (d *ddl) dropTableObject( + ctx sessionctx.Context, + objects []*ast.TableName, + ifExists bool, + tableObjectType objectType, +) error { + var ( + notExistTables []string + sessVars = ctx.GetSessionVars() + is = d.GetInfoSchemaWithInterceptor(ctx) + dropExistErr *terror.Error + jobType model.ActionType + ) + + switch tableObjectType { + case tableObject: + dropExistErr = infoschema.ErrTableDropExists + jobType = model.ActionDropTable + case viewObject: + dropExistErr = infoschema.ErrTableDropExists + jobType = model.ActionDropView + case sequenceObject: + dropExistErr = infoschema.ErrSequenceDropExists + jobType = model.ActionDropSequence + } + + for _, tn := range objects { + fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} + schema, ok := is.SchemaByName(tn.Schema) + if !ok { + // TODO: we should return special error for table not exist, checking "not exist" is not enough, + // because some other errors may contain this error string too. + notExistTables = append(notExistTables, fullti.String()) + continue + } + tableInfo, err := is.TableByName(tn.Schema, tn.Name) + if err != nil && infoschema.ErrTableNotExists.Equal(err) { + notExistTables = append(notExistTables, fullti.String()) + continue + } else if err != nil { + return err + } + + // prechecks before build DDL job + + // Protect important system table from been dropped by a mistake. + // I can hardly find a case that a user really need to do this. + 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) + } + switch tableObjectType { + case tableObject: + if !tableInfo.Meta().IsBaseTable() { + notExistTables = append(notExistTables, fullti.String()) + continue + } + + tempTableType := tableInfo.Meta().TempTableType + if 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), + ) + exec := ctx.(sqlexec.RestrictedSQLExecutor) + _, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) + if err != nil { + return err + } + } + + if tableInfo.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Table") + } + case viewObject: + if !tableInfo.Meta().IsView() { + return dbterror.ErrWrongObject.GenWithStackByArgs(fullti.Schema, fullti.Name, "VIEW") + } + case sequenceObject: + if !tableInfo.Meta().IsSequence() { + err = dbterror.ErrWrongObject.GenWithStackByArgs(fullti.Schema, fullti.Name, "SEQUENCE") + if ifExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + continue + } + return err + } + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: tableInfo.Meta().ID, + SchemaName: schema.Name.L, + SchemaState: schema.State, + TableName: tableInfo.Meta().Name.L, + Type: jobType, + BinlogInfo: &model.HistoryInfo{}, + } + + err = d.DoDDLJob(ctx, job) + err = d.callHookOnChanged(job, err) + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { + notExistTables = append(notExistTables, fullti.String()) + continue + } else if err != nil { + return errors.Trace(err) + } + + // unlock table after drop + if tableObjectType != tableObject { + continue + } + if !config.TableLockEnabled() { + continue + } + if ok, _ := ctx.CheckTableLocked(tableInfo.Meta().ID); ok { + ctx.ReleaseTableLockByTableIDs([]int64{tableInfo.Meta().ID}) + } - err = d.DoDDLJob(ctx, job) - err = d.callHookOnChanged(job, err) - if err != nil { - return errors.Trace(err) } - if !config.TableLockEnabled() { - return nil + if len(notExistTables) > 0 && !ifExists { + return dropExistErr.GenWithStackByArgs(strings.Join(notExistTables, ",")) } - if ok, _ := ctx.CheckTableLocked(tb.Meta().ID); ok { - ctx.ReleaseTableLockByTableIDs([]int64{tb.Meta().ID}) + // We need add warning when use if exists. + if len(notExistTables) > 0 && ifExists { + for _, table := range notExistTables { + sessVars.StmtCtx.AppendNote(dropExistErr.GenWithStackByArgs(table)) + } } return nil } -// DropView will proceed even if some view in the list does not exists. -func (d *ddl) DropView(ctx sessionctx.Context, ti ast.Ident) (err error) { - schema, tb, err := d.getSchemaAndTableByIdent(ctx, ti) - if err != nil { - return errors.Trace(err) - } - - if !tb.Meta().IsView() { - return dbterror.ErrWrongObject.GenWithStackByArgs(ti.Schema, ti.Name, "VIEW") - } - - job := &model.Job{ - SchemaID: schema.ID, - TableID: tb.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: tb.Meta().State, - TableName: tb.Meta().Name.L, - Type: model.ActionDropView, - BinlogInfo: &model.HistoryInfo{}, - } +// DropTable will proceed even if some table in the list does not exists. +func (d *ddl) DropTable(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) { + return d.dropTableObject(ctx, stmt.Tables, stmt.IfExists, tableObject) +} - err = d.DoDDLJob(ctx, job) - err = d.callHookOnChanged(job, err) - return errors.Trace(err) +// DropView will proceed even if some view in the list does not exists. +func (d *ddl) DropView(ctx sessionctx.Context, stmt *ast.DropTableStmt) (err error) { + return d.dropTableObject(ctx, stmt.Tables, stmt.IfExists, viewObject) } func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { @@ -6676,34 +6777,8 @@ func (d *ddl) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) return errors.Trace(err) } -func (d *ddl) DropSequence(ctx sessionctx.Context, ti ast.Ident, ifExists bool) (err error) { - schema, tbl, err := d.getSchemaAndTableByIdent(ctx, ti) - if err != nil { - return errors.Trace(err) - } - - if !tbl.Meta().IsSequence() { - err = dbterror.ErrWrongObject.GenWithStackByArgs(ti.Schema, ti.Name, "SEQUENCE") - if ifExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil - } - return err - } - - job := &model.Job{ - SchemaID: schema.ID, - TableID: tbl.Meta().ID, - SchemaName: schema.Name.L, - SchemaState: tbl.Meta().State, - TableName: tbl.Meta().Name.L, - Type: model.ActionDropSequence, - BinlogInfo: &model.HistoryInfo{}, - } - - err = d.DoDDLJob(ctx, job) - err = d.callHookOnChanged(job, err) - return errors.Trace(err) +func (d *ddl) DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceStmt) (err error) { + return d.dropTableObject(ctx, stmt.Sequences, stmt.IfExists, sequenceObject) } func (d *ddl) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName model.CIStr, visibility ast.IndexVisibility) error { diff --git a/executor/ddl.go b/executor/ddl.go index 1553be2299fd7..a3015e5fd069e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" ) @@ -335,117 +334,16 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { return err } -// If one drop those tables by mistake, it's difficult to recover. -// In the worst case, the whole TiDB cluster fails to bootstrap, so we prevent user from dropping them. -var systemTables = map[string]struct{}{ - "tidb": {}, - "gc_delete_range": {}, - "gc_delete_range_done": {}, -} - -func isSystemTable(schema, table string) bool { - if schema != "mysql" { - return false - } - if _, ok := systemTables[table]; ok { - return true - } - return false -} - -type objectType int - -const ( - tableObject objectType = iota - viewObject - sequenceObject -) - func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { - return e.dropTableObject(s.Tables, tableObject, s.IfExists) + return domain.GetDomain(e.ctx).DDL().DropTable(e.ctx, s) } func (e *DDLExec) executeDropView(s *ast.DropTableStmt) error { - return e.dropTableObject(s.Tables, viewObject, s.IfExists) + return domain.GetDomain(e.ctx).DDL().DropView(e.ctx, s) } func (e *DDLExec) executeDropSequence(s *ast.DropSequenceStmt) error { - return e.dropTableObject(s.Sequences, sequenceObject, s.IfExists) -} - -// dropTableObject actually applies to `tableObject`, `viewObject` and `sequenceObject`. -func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifExists bool) error { - var notExistTables []string - sessVars := e.ctx.GetSessionVars() - for _, tn := range objects { - fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} - _, ok := e.is.SchemaByName(tn.Schema) - if !ok { - // TODO: we should return special error for table not exist, checking "not exist" is not enough, - // because some other errors may contain this error string too. - notExistTables = append(notExistTables, fullti.String()) - continue - } - _, err := e.is.TableByName(tn.Schema, tn.Name) - if err != nil && infoschema.ErrTableNotExists.Equal(err) { - notExistTables = append(notExistTables, fullti.String()) - continue - } else if err != nil { - return err - } - - // Protect important system table from been dropped by a mistake. - // I can hardly find a case that a user really need to do this. - 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) - } - 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), - ) - exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - _, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) - if err != nil { - return err - } - } - switch obt { - case tableObject: - err = domain.GetDomain(e.ctx).DDL().DropTable(e.ctx, fullti) - case viewObject: - err = domain.GetDomain(e.ctx).DDL().DropView(e.ctx, fullti) - case sequenceObject: - err = domain.GetDomain(e.ctx).DDL().DropSequence(e.ctx, fullti, ifExists) - } - if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { - notExistTables = append(notExistTables, fullti.String()) - } else if err != nil { - return err - } - } - if len(notExistTables) > 0 && !ifExists { - if obt == sequenceObject { - return infoschema.ErrSequenceDropExists.GenWithStackByArgs(strings.Join(notExistTables, ",")) - } - return infoschema.ErrTableDropExists.GenWithStackByArgs(strings.Join(notExistTables, ",")) - } - // We need add warning when use if exists. - if len(notExistTables) > 0 && ifExists { - for _, table := range notExistTables { - if obt == sequenceObject { - sessVars.StmtCtx.AppendNote(infoschema.ErrSequenceDropExists.GenWithStackByArgs(table)) - } else { - sessVars.StmtCtx.AppendNote(infoschema.ErrTableDropExists.GenWithStackByArgs(table)) - } - } - } - return nil + return domain.GetDomain(e.ctx).DDL().DropSequence(e.ctx, s) } func (e *DDLExec) dropLocalTemporaryTables(localTempTables []*ast.TableName) error { From c5ad449d1ae2f1e41ef653fe888cb218c0415048 Mon Sep 17 00:00:00 2001 From: you06 Date: Wed, 29 Jun 2022 10:54:39 +0800 Subject: [PATCH 16/28] metrics: add loading region cache duration (#34679) ref pingcap/tidb#34106 --- metrics/grafana/tidb.json | 373 +++++++++++++++++++++++++++++--------- 1 file changed, 284 insertions(+), 89 deletions(-) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 30d2c788dc0e1..333ac46163b73 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -8387,6 +8387,288 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB successful region cache operations count", + "fill": 1, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 15 + }, + "id": 164, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result=\"ok\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 30 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Region Cache OK OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB error region cache operations count", + "fill": 1, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 22 + }, + "id": 250, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result=\"err\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-err", + "refId": "A", + "step": 30 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Region Cache Error OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB loading region cache durations", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 22 + }, + "hiddenSeries": false, + "id": 251, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_load_region_cache_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "99-{{type}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_tikvclient_load_region_cache_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type) / sum(rate(tidb_tikvclient_load_region_cache_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-{{type}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Load Region Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -12818,93 +13100,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB region cache operations count", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 21 - }, - "id": 164, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", result=\"err\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Region Cache Error OPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, { "aliasColors": {}, "bars": false, @@ -12916,8 +13111,8 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 21 + "x": 0, + "y": 42 }, "id": 52, "legend": { From 5d7915c6f188c5afb5494c88430f536c77fe3249 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 29 Jun 2022 11:30:39 +0800 Subject: [PATCH 17/28] tools/check: skip 'tests' directory in 'make ut' (#35800) close pingcap/tidb#35799 --- tools/check/ut.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/check/ut.go b/tools/check/ut.go index 84b00d34d9e60..66a4fac7203a8 100644 --- a/tools/check/ut.go +++ b/tools/check/ut.go @@ -819,7 +819,7 @@ func (n *numa) testCommand(pkg string, fn string) *exec.Cmd { } func skipDIR(pkg string) bool { - skipDir := []string{"br", "cmd", "dumpling"} + skipDir := []string{"br", "cmd", "dumpling", "tests"} for _, ignore := range skipDir { if strings.HasPrefix(pkg, ignore) { return true From c6103886b15fd83511ef99d94fe2eb4d71b0cd1d Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 29 Jun 2022 11:54:39 +0800 Subject: [PATCH 18/28] *: enable all gosimples (#35762) ref pingcap/tidb#35345 --- br/pkg/lightning/backend/local/local.go | 1 - br/pkg/lightning/config/config.go | 2 +- br/pkg/lightning/restore/restore.go | 1 - build/BUILD.bazel | 24 ++++ build/nogo_config.json | 182 ++++++++++++++++++++++++ ddl/ddl_api.go | 3 - executor/brie.go | 1 - executor/merge_join_test.go | 18 +-- expression/builtin_compare.go | 18 +-- infoschema/cluster.go | 16 +-- parser/ast/misc.go | 1 - parser/lexer.go | 4 +- parser/model/ddl.go | 4 +- parser/types/etc.go | 8 +- util/column-mapping/column.go | 2 +- 15 files changed, 237 insertions(+), 48 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 8a45472dddfcc..6679d8bf93b98 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1504,7 +1504,6 @@ func (local *local) ResolveDuplicateRows(ctx context.Context, tbl table.Table, t logger.Warn("[resolve-dupe] skipping resolution due to selected algorithm. this table will become inconsistent!", zap.Stringer("algorithm", algorithm)) return nil case config.DupeResAlgRemove: - break default: panic(fmt.Sprintf("[resolve-dupe] unknown resolution algorithm %v", algorithm)) } diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index b0ffe32fa3cd5..0066895568550 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -1156,7 +1156,7 @@ func (cfg *Config) CheckAndAdjustSecurity() error { return common.ErrInvalidConfig.GenWithStack("cannot set `tidb.tls` to 'cluster' without a [security] section") } case "false", "skip-verify", "preferred": - break + return nil default: return common.ErrInvalidConfig.GenWithStack("unsupported `tidb.tls` config %s", cfg.TiDB.TLS) } diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 6246c27ef411b..faf774efa5c7a 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -916,7 +916,6 @@ func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string switch { case err == nil: - break case utils.MessageIsRetryableStorageError(err.Error()), common.IsContextCanceledError(err): // recoverable error, should not be recorded in checkpoint // which will prevent lightning from automatically recovering diff --git a/build/BUILD.bazel b/build/BUILD.bazel index 6957be918b42b..b283c4b2e0f20 100644 --- a/build/BUILD.bazel +++ b/build/BUILD.bazel @@ -4,17 +4,41 @@ load("@io_bazel_rules_go//go:def.bzl", "nogo") load("//build/linter/staticcheck:def.bzl", "staticcheck_analyzers") STATICHECK_ANALYZERS = [ + "S1000", + "S1001", "S1002", + "S1003", "S1004", + "S1005", + "S1006", "S1007", + "S1008", "S1009", "S1010", + "S1011", "S1012", + "S1016", + "S1017", + "S1018", "S1019", "S1020", "S1021", + "S1023", "S1024", + "S1025", + "S1028", + "S1029", "S1030", + "S1031", + "S1032", + "S1033", + "S1034", + "S1035", + "S1036", + "S1037", + "S1038", + "S1039", + "S1040", "SA2000", "SA2001", "SA2003", diff --git a/build/nogo_config.json b/build/nogo_config.json index cefdb5fe4aa11..8f7f4e940a597 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -230,24 +230,60 @@ "parser/digester_test.go": "ignore code" } }, + "S1000": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1001": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1002": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1003": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1004": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1005": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1006": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1007": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1008": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1009": { "exclude_files": { "/external/": "no need to vet third party code", @@ -260,12 +296,54 @@ ".*_generated\\.go$": "ignore generated code" } }, + "S1011": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1012": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1013": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1014": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1015": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1016": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1017": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1018": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1019": { "exclude_files": { "/external/": "no need to vet third party code", @@ -286,18 +364,122 @@ "tools/check/ut.go": "ignore code" } }, + "S1022": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1023": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "parser/parser.go": "ignore code" + } + }, "S1024": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1025": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1026": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1027": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1028": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1029": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "S1030": { "exclude_files": { "/external/": "no need to vet third party code", ".*_generated\\.go$": "ignore generated code" } }, + "S1031": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1032": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1033": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1034": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1035": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1036": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1037": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1038": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1039": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1040": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "parser/parser.go": "ignore generated code" + } + }, "SA2000": { "exclude_files": { "/external/": "no need to vet third party code", diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 23a0de2c32eb6..d0b799731a779 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4706,7 +4706,6 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe } switch { case oldRandBits == newRandBits: - break case oldRandBits < newRandBits: addingAutoRandom := oldRandBits == 0 if addingAutoRandom { @@ -6341,7 +6340,6 @@ func validateCommentLength(vars *variable.SessionVars, name string, comment *str case dbterror.ErrTooLongTableComment: maxLen *= 2 case dbterror.ErrTooLongFieldComment, dbterror.ErrTooLongIndexComment, dbterror.ErrTooLongTablePartitionComment: - break default: // add more types of terror.Error if need } @@ -6405,7 +6403,6 @@ func checkColumnsTypeAndValuesMatch(ctx sessionctx.Context, meta *model.TableInf case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: switch vkind { case types.KindString, types.KindBytes: - break default: return dbterror.ErrWrongTypeColumnValue.GenWithStackByArgs() } diff --git a/executor/brie.go b/executor/brie.go index 3cd1ce7ea9e1e..78fdcbe9f1285 100644 --- a/executor/brie.go +++ b/executor/brie.go @@ -249,7 +249,6 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema) return nil } default: - break } if tidbCfg.Store != "tikv" { diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index a140809fd46d2..2a46243c6e6cc 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -785,11 +785,11 @@ func TestVectorizedMergeJoin(t *testing.T) { )).Check(testkit.Rows( fmt.Sprintf(`MergeJoin 4150.01 root inner join, left key:test.%s.a, right key:test.%s.a`, t1, t2), fmt.Sprintf(`├─Sort(Build) 3320.01 root test.%s.a`, t2), - fmt.Sprintf(`│ └─TableReader 3320.01 root data:Selection`), + `│ └─TableReader 3320.01 root data:Selection`, fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), fmt.Sprintf(`└─Sort(Probe) 3330.00 root test.%s.a`, t1), - fmt.Sprintf(` └─TableReader 3330.00 root data:Selection`), + ` └─TableReader 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) @@ -797,10 +797,10 @@ func TestVectorizedMergeJoin(t *testing.T) { t1, t2, t1, t2, t1, t2, t1, t2, )).Check(testkit.Rows( fmt.Sprintf(`HashJoin 4150.01 root inner join, equal:[eq(test.%s.a, test.%s.a)]`, t1, t2), - fmt.Sprintf(`├─TableReader(Build) 3320.01 root data:Selection`), + `├─TableReader(Build) 3320.01 root data:Selection`, fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), - fmt.Sprintf(`└─TableReader(Probe) 3330.00 root data:Selection`), + `└─TableReader(Probe) 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) @@ -903,14 +903,14 @@ func TestVectorizedShuffleMergeJoin(t *testing.T) { tk.MustQuery(fmt.Sprintf("explain format = 'brief' select /*+ TIDB_SMJ(%s, %s) */ * from %s, %s where %s.a=%s.a and %s.b>5 and %s.b<5", t1, t2, t1, t2, t1, t2, t1, t2, )).Check(testkit.Rows( - fmt.Sprintf(`Shuffle 4150.01 root execution info: concurrency:4, data sources:[TableReader TableReader]`), + `Shuffle 4150.01 root execution info: concurrency:4, data sources:[TableReader TableReader]`, fmt.Sprintf(`└─MergeJoin 4150.01 root inner join, left key:test.%s.a, right key:test.%s.a`, t1, t2), fmt.Sprintf(` ├─Sort(Build) 3320.01 root test.%s.a`, t2), - fmt.Sprintf(` │ └─TableReader 3320.01 root data:Selection`), + ` │ └─TableReader 3320.01 root data:Selection`, fmt.Sprintf(` │ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(` │ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), fmt.Sprintf(` └─Sort(Probe) 3330.00 root test.%s.a`, t1), - fmt.Sprintf(` └─TableReader 3330.00 root data:Selection`), + ` └─TableReader 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) @@ -918,10 +918,10 @@ func TestVectorizedShuffleMergeJoin(t *testing.T) { t1, t2, t1, t2, t1, t2, t1, t2, )).Check(testkit.Rows( fmt.Sprintf(`HashJoin 4150.01 root inner join, equal:[eq(test.%s.a, test.%s.a)]`, t1, t2), - fmt.Sprintf(`├─TableReader(Build) 3320.01 root data:Selection`), + `├─TableReader(Build) 3320.01 root data:Selection`, fmt.Sprintf(`│ └─Selection 3320.01 cop[tikv] lt(test.%s.b, 5), not(isnull(test.%s.a))`, t2, t2), fmt.Sprintf(`│ └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t2), - fmt.Sprintf(`└─TableReader(Probe) 3330.00 root data:Selection`), + `└─TableReader(Probe) 3330.00 root data:Selection`, fmt.Sprintf(` └─Selection 3330.00 cop[tikv] gt(test.%s.b, 5), not(isnull(test.%s.a))`, t1, t1), fmt.Sprintf(` └─TableFullScan 10000.00 cop[tikv] table:%s keep order:false, stats:pseudo`, t1), )) diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 3fc5c9a58adf0..6ef85a4b9c057 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -2540,21 +2540,21 @@ func (b *builtinNullEQIntSig) evalInt(row chunk.Row) (val int64, isNull bool, er case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case isUnsigned0 && isUnsigned1 && types.CompareUint64(uint64(arg0), uint64(arg1)) == 0: res = 1 case !isUnsigned0 && !isUnsigned1 && types.CompareInt64(arg0, arg1) == 0: res = 1 case isUnsigned0 && !isUnsigned1: if arg1 < 0 { - break + return res, false, nil } if types.CompareInt64(arg0, arg1) == 0 { res = 1 } case !isUnsigned0 && isUnsigned1: if arg0 < 0 { - break + return res, false, nil } if types.CompareInt64(arg0, arg1) == 0 { res = 1 @@ -2587,7 +2587,7 @@ func (b *builtinNullEQRealSig) evalInt(row chunk.Row) (val int64, isNull bool, e case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case types.CompareFloat64(arg0, arg1) == 0: res = 1 } @@ -2618,7 +2618,7 @@ func (b *builtinNullEQDecimalSig) evalInt(row chunk.Row) (val int64, isNull bool case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case arg0.Compare(arg1) == 0: res = 1 } @@ -2649,7 +2649,7 @@ func (b *builtinNullEQStringSig) evalInt(row chunk.Row) (val int64, isNull bool, case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case types.CompareString(arg0, arg1, b.collation) == 0: res = 1 } @@ -2680,7 +2680,7 @@ func (b *builtinNullEQDurationSig) evalInt(row chunk.Row) (val int64, isNull boo case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case arg0.Compare(arg1) == 0: res = 1 } @@ -2711,7 +2711,7 @@ func (b *builtinNullEQTimeSig) evalInt(row chunk.Row) (val int64, isNull bool, e case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil case arg0.Compare(arg1) == 0: res = 1 } @@ -2742,7 +2742,7 @@ func (b *builtinNullEQJSONSig) evalInt(row chunk.Row) (val int64, isNull bool, e case isNull0 && isNull1: res = 1 case isNull0 != isNull1: - break + return res, false, nil default: cmpRes := json.CompareBinary(arg0, arg1) if cmpRes == 0 { diff --git a/infoschema/cluster.go b/infoschema/cluster.go index d1d4f3c5a7fac..47bb7db4c3434 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -79,16 +79,14 @@ func isClusterTableByName(dbName, tableName string) bool { dbName = strings.ToUpper(dbName) switch dbName { case util.InformationSchemaName.O, util.PerformanceSchemaName.O: - break - default: - return false - } - tableName = strings.ToUpper(tableName) - for _, name := range memTableToClusterTables { - name = strings.ToUpper(name) - if name == tableName { - return true + tableName = strings.ToUpper(tableName) + for _, name := range memTableToClusterTables { + name = strings.ToUpper(name) + if name == tableName { + return true + } } + default: } return false } diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 3ec7d5c753ba8..38c465568634a 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -619,7 +619,6 @@ const ( func (n CompletionType) Restore(ctx *format.RestoreCtx) error { switch n { case CompletionTypeDefault: - break case CompletionTypeChain: ctx.WriteKeyWord(" AND CHAIN") case CompletionTypeRelease: diff --git a/parser/lexer.go b/parser/lexer.go index 659fc14dd9235..bcddc5f5bfcea 100644 --- a/parser/lexer.go +++ b/parser/lexer.go @@ -478,7 +478,6 @@ func startWithSlash(s *Scanner) (tok int, pos Pos, lit string) { } case 'M': // '/*M' maybe MariaDB-specific comments // no special treatment for now. - break case '+': // '/*+' optimizer hints // See https://dev.mysql.com/doc/refman/5.7/en/optimizer-hints.html @@ -502,7 +501,6 @@ func startWithSlash(s *Scanner) (tok int, pos Pos, lit string) { currentCharIsStar = true default: - break } // standard C-like comment. read until we see '*/' then drop it. @@ -574,7 +572,7 @@ func startWithAt(s *Scanner) (tok int, pos Pos, lit string) { tok, lit = doubleAtIdentifier, s.r.data(&pos) } case invalid: - break + return default: tok = singleAtIdentifier } diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 549a8119e6b33..993c2c485f50c 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -202,9 +202,7 @@ func (h *HistoryInfo) AddTableInfo(schemaVer int64, tblInfo *TableInfo) { func (h *HistoryInfo) SetTableInfos(schemaVer int64, tblInfos []*TableInfo) { h.SchemaVersion = schemaVer h.MultipleTableInfos = make([]*TableInfo, len(tblInfos)) - for i, info := range tblInfos { - h.MultipleTableInfos[i] = info - } + copy(h.MultipleTableInfos, tblInfos) } // Clean cleans history information. diff --git a/parser/types/etc.go b/parser/types/etc.go index 2fe3d113e8820..1fdfeaf05367f 100644 --- a/parser/types/etc.go +++ b/parser/types/etc.go @@ -128,12 +128,8 @@ func TypeToStr(tp byte, cs string) (r string) { // Args: // ts: type string func StrToType(ts string) (tp byte) { - if strings.Contains(ts, "blob") { - ts = strings.Replace(ts, "blob", "text", 1) - } else if strings.Contains(ts, "binary") { - ts = strings.Replace(ts, "binary", "char", 1) - } - + ts = strings.Replace(ts, "blob", "text", 1) + ts = strings.Replace(ts, "binary", "char", 1) if tp, ok := str2Type[ts]; ok { return tp } diff --git a/util/column-mapping/column.go b/util/column-mapping/column.go index e276b0004882c..93a1443dc6173 100644 --- a/util/column-mapping/column.go +++ b/util/column-mapping/column.go @@ -112,7 +112,7 @@ func (r *Rule) Valid() error { if r.Expression == PartitionID { switch len(r.Arguments) { case 3, 4: - break + return nil default: return errors.NotValidf("arguments %v for patition id", r.Arguments) } From 5e547b5549ad764499cf8bdfeec4e2686cb93d2a Mon Sep 17 00:00:00 2001 From: Yujie Xia Date: Wed, 29 Jun 2022 12:46:39 +0800 Subject: [PATCH 19/28] lightning: avoid extracting db schema when schema file is not provided (#35783) close pingcap/tidb#35763 --- br/pkg/lightning/mydump/loader.go | 26 ++++++++++++-------------- br/pkg/lightning/mydump/loader_test.go | 8 ++++++-- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/br/pkg/lightning/mydump/loader.go b/br/pkg/lightning/mydump/loader.go index 30f4f14c1464c..40091c61b2d03 100644 --- a/br/pkg/lightning/mydump/loader.go +++ b/br/pkg/lightning/mydump/loader.go @@ -39,21 +39,19 @@ type MDDatabaseMeta struct { } func (m *MDDatabaseMeta) GetSchema(ctx context.Context, store storage.ExternalStorage) string { - schema, err := ExportStatement(ctx, store, m.SchemaFile, m.charSet) - if err != nil { - log.FromContext(ctx).Warn("failed to extract table schema", - zap.String("Path", m.SchemaFile.FileMeta.Path), - log.ShortError(err), - ) - schema = nil - } - schemaStr := strings.TrimSpace(string(schema)) - // set default if schema sql is empty - if len(schemaStr) == 0 { - schemaStr = "CREATE DATABASE IF NOT EXISTS " + common.EscapeIdentifier(m.Name) + if m.SchemaFile.FileMeta.Path != "" { + schema, err := ExportStatement(ctx, store, m.SchemaFile, m.charSet) + if err != nil { + log.FromContext(ctx).Warn("failed to extract table schema", + zap.String("Path", m.SchemaFile.FileMeta.Path), + log.ShortError(err), + ) + } else if schemaStr := strings.TrimSpace(string(schema)); schemaStr != "" { + return schemaStr + } } - - return schemaStr + // set default if schema sql is empty or failed to extract. + return "CREATE DATABASE IF NOT EXISTS " + common.EscapeIdentifier(m.Name) } type MDTableMeta struct { diff --git a/br/pkg/lightning/mydump/loader_test.go b/br/pkg/lightning/mydump/loader_test.go index 0f7079a1f95df..e256a685f20ee 100644 --- a/br/pkg/lightning/mydump/loader_test.go +++ b/br/pkg/lightning/mydump/loader_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/log" md "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/storage" filter "github.com/pingcap/tidb/util/table-filter" @@ -181,13 +182,16 @@ func TestTableInfoNotFound(t *testing.T) { loader, err := md.NewMyDumpLoader(ctx, s.cfg) require.NoError(t, err) for _, dbMeta := range loader.GetDatabases() { - dbSQL := dbMeta.GetSchema(ctx, store) + logger, buffer := log.MakeTestLogger() + logCtx := log.NewContext(ctx, logger) + dbSQL := dbMeta.GetSchema(logCtx, store) require.Equal(t, "CREATE DATABASE IF NOT EXISTS `db`", dbSQL) for _, tblMeta := range dbMeta.Tables { - sql, err := tblMeta.GetSchema(ctx, store) + sql, err := tblMeta.GetSchema(logCtx, store) require.Equal(t, "", sql) require.NoError(t, err) } + require.NotContains(t, buffer.Stripped(), "failed to extract table schema") } } From 41c0ab4879d8d5918f616443a1e2be4fb1da4ca3 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 28 Jun 2022 23:12:39 -0600 Subject: [PATCH 20/28] *: support canceling DDL statements with KILL (#35803) close pingcap/tidb#24144 --- ddl/ddl.go | 5 +++++ executor/ddl.go | 5 ++++- server/server.go | 20 ++++++++++++++++++++ sessionctx/stmtctx/stmtctx.go | 1 + 4 files changed, 30 insertions(+), 1 deletion(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index c86b8f60e7459..81c0d01bb3d52 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -737,6 +737,11 @@ func (d *ddl) DoDDLJob(ctx sessionctx.Context, job *model.Job) error { var historyJob *model.Job jobID := job.ID + + // Attach the context of the jobId to the calling session so that + // KILL can cancel this DDL job. + ctx.GetSessionVars().StmtCtx.DDLJobID = jobID + // For a job from start to end, the state of it will be none -> delete only -> write only -> reorganization -> public // For every state changes, we will wait as lease 2 * lease time, so here the ticker check is 10 * lease. // But we use etcd to speed up, normally it takes less than 0.5s now, so we use 0.5s or 1s or 3s as the max value. diff --git a/executor/ddl.go b/executor/ddl.go index a3015e5fd069e..68115f2014e44 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -132,7 +132,10 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { return err } - defer func() { e.ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = false }() + defer func() { + e.ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = false + e.ctx.GetSessionVars().StmtCtx.DDLJobID = 0 + }() switch x := e.stmt.(type) { case *ast.AlterDatabaseStmt: diff --git a/server/server.go b/server/server.go index 13ed052391f59..63406905b5a3b 100644 --- a/server/server.go +++ b/server/server.go @@ -50,6 +50,7 @@ import ( "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" @@ -718,6 +719,25 @@ func killConn(conn *clientConn) { conn.mu.RLock() cancelFunc := conn.mu.cancelFunc conn.mu.RUnlock() + + // If the connection being killed is a DDL Job, + // we need to CANCEL the matching jobID first. + if sessVars.StmtCtx.IsDDLJobInQueue { + jobID := sessVars.StmtCtx.DDLJobID + err := kv.RunInNewTxn(context.Background(), conn.ctx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { + // errs is the error per job, there is only one submitted + // err is the error of the overall task + errs, err := ddl.CancelJobs(txn, []int64{jobID}) + if len(errs) > 0 { + logutil.BgLogger().Warn("error canceling DDL job", zap.Error(errs[0])) + } + return err + }) + if err != nil { + logutil.BgLogger().Warn("could not cancel DDL job", zap.Error(err)) + } + } + if cancelFunc != nil { cancelFunc() } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 4d623015492cc..6d5041d2cdfbf 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -109,6 +109,7 @@ type StatementContext struct { // IsDDLJobInQueue is used to mark whether the DDL job is put into the queue. // If IsDDLJobInQueue is true, it means the DDL job is in the queue of storage, and it can be handled by the DDL worker. IsDDLJobInQueue bool + DDLJobID int64 InInsertStmt bool InUpdateStmt bool InDeleteStmt bool From 2874911542dc8799af39c019b73fc29c637f7116 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Wed, 29 Jun 2022 13:30:39 +0800 Subject: [PATCH 21/28] txn: fix the error message if retry could not be used for rc check ts (#35474) close pingcap/tidb#35473 --- sessiontxn/isolation/readcommitted.go | 2 +- store/copr/coprocessor.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 5fb316b59f8bf..985b6a04800f3 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -168,7 +168,7 @@ func (p *PessimisticRCTxnContextProvider) handleAfterQueryError(queryErr error) p.latestOracleTSValid = false logutil.Logger(p.ctx).Info("RC read with ts checking has failed, retry RC read", - zap.String("sql", sessVars.StmtCtx.OriginalSQL)) + zap.String("sql", sessVars.StmtCtx.OriginalSQL), zap.Error(queryErr)) return sessiontxn.RetryReady() } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 0b217583bb9b3..828d3546abfae 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -934,7 +934,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R zap.String("storeAddr", task.storeAddr), zap.Error(err)) if strings.Contains(err.Error(), "write conflict") { - return nil, kv.ErrWriteConflict + return nil, kv.ErrWriteConflict.FastGen("%s", otherErr) } return nil, errors.Trace(err) } From f0d5f6e9cd1140113dcc033ace7ec19e207408fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Wed, 29 Jun 2022 13:52:39 +0800 Subject: [PATCH 22/28] txn: remove `legacy.SimpleTxnContextProvider` (#35667) close pingcap/tidb#35666 --- executor/BUILD.bazel | 1 - executor/builder.go | 85 ++-------- executor/stale_txn_test.go | 2 - session/BUILD.bazel | 1 - session/session.go | 10 +- sessionctx/sessionstates/BUILD.bazel | 1 + sessionctx/stmtctx/BUILD.bazel | 4 + sessiontxn/isolation/BUILD.bazel | 1 + sessiontxn/legacy/BUILD.bazel | 41 ----- sessiontxn/legacy/provider.go | 243 --------------------------- sessiontxn/legacy/provider_test.go | 126 -------------- sessiontxn/txn_context_test.go | 2 - 12 files changed, 23 insertions(+), 494 deletions(-) delete mode 100644 sessiontxn/legacy/BUILD.bazel delete mode 100644 sessiontxn/legacy/provider.go delete mode 100644 sessiontxn/legacy/provider_test.go diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index e9b7c23298645..94f32782965ed 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -132,7 +132,6 @@ go_library( "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", - "//sessiontxn/legacy", "//sessiontxn/staleread", "//statistics", "//statistics/handle", diff --git a/executor/builder.go b/executor/builder.go index db0acf8eb02aa..ba5cebc836363 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/legacy" "github.com/pingcap/tidb/sessiontxn/staleread" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/helper" @@ -88,14 +87,11 @@ var ( // executorBuilder builds an Executor from a Plan. // The InfoSchema must not change during execution. type executorBuilder struct { - ctx sessionctx.Context - is infoschema.InfoSchema - snapshotTS uint64 // The ts for snapshot-read. A select statement without for update will use this ts - forUpdateTS uint64 // The ts should be used by insert/update/delete/select-for-update statement - snapshotTSCached bool - err error // err is set when there is error happened during Executor building process. - hasLock bool - Ti *TelemetryInfo + ctx sessionctx.Context + is infoschema.InfoSchema + err error // err is set when there is error happened during Executor building process. + hasLock bool + Ti *TelemetryInfo // isStaleness means whether this statement use stale read. isStaleness bool readReplicaScope string @@ -121,26 +117,13 @@ type CTEStorages struct { } func newExecutorBuilder(ctx sessionctx.Context, is infoschema.InfoSchema, ti *TelemetryInfo, replicaReadScope string) *executorBuilder { - b := &executorBuilder{ + return &executorBuilder{ ctx: ctx, is: is, Ti: ti, isStaleness: staleread.IsStmtStaleness(ctx), readReplicaScope: replicaReadScope, } - - txnManager := sessiontxn.GetTxnManager(ctx) - if provider, ok := txnManager.GetContextProvider().(*legacy.SimpleTxnContextProvider); ok { - provider.GetReadTSFunc = b.getReadTS - provider.GetForUpdateTSFunc = func() (uint64, error) { - if b.forUpdateTS != 0 { - return b.forUpdateTS, nil - } - return b.getReadTS() - } - } - - return b } // MockPhysicalPlan is used to return a specified executor in when build. @@ -657,9 +640,7 @@ func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor defer func() { b.inSelectLockStmt = false }() } b.hasLock = true - - // Build 'select for update' using the 'for update' ts. - if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } @@ -865,8 +846,7 @@ func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { b.inInsertStmt = true - - if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } @@ -1581,44 +1561,6 @@ func (b *executorBuilder) getSnapshotTS() (uint64, error) { return txnManager.GetStmtReadTS() } -// getReadTS returns the ts used by select (without for-update clause). The return value is affected by the isolation level -// and some stale/historical read contexts. For example, it will return txn.StartTS in RR and return -// the current timestamp in RC isolation -func (b *executorBuilder) getReadTS() (uint64, error) { - failpoint.Inject("assertNotStaleReadForExecutorGetReadTS", func() { - // after refactoring stale read will use its own context provider - staleread.AssertStmtStaleness(b.ctx, false) - }) - - if b.snapshotTSCached { - return b.snapshotTS, nil - } - - if snapshotTS := b.ctx.GetSessionVars().SnapshotTS; snapshotTS != 0 { - b.snapshotTS = snapshotTS - b.snapshotTSCached = true - return snapshotTS, nil - } - - if b.snapshotTS != 0 { - b.snapshotTSCached = true - // Return the cached value. - return b.snapshotTS, nil - } - - txn, err := b.ctx.Txn(true) - if err != nil { - return 0, err - } - - b.snapshotTS = txn.StartTS() - if b.snapshotTS == 0 { - return 0, errors.Trace(ErrGetStartTS) - } - b.snapshotTSCached = true - return b.snapshotTS, nil -} - func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executor { switch v.DBName.L { case util.MetricSchemaName.L: @@ -2119,8 +2061,7 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { } } } - - if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } @@ -2178,7 +2119,7 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { tblID2table[info.TblID], _ = b.is.TableByID(info.TblID) } - if b.forUpdateTS, b.err = b.getSnapshotTS(); b.err != nil { + if b.err = b.updateForUpdateTS(); b.err != nil { return nil } @@ -2197,6 +2138,12 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { return deleteExec } +func (b *executorBuilder) updateForUpdateTS() error { + // GetStmtForUpdateTS will auto update the for update ts if it is necessary + _, err := sessiontxn.GetTxnManager(b.ctx).GetStmtForUpdateTS() + return err +} + func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O} _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 62255326cbce0..51e50b670ade2 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -34,10 +34,8 @@ import ( func enableStaleReadCommonFailPoint(t *testing.T) func() { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) return func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/assertStaleReadForOptimizePreparedPlan")) } } diff --git a/session/BUILD.bazel b/session/BUILD.bazel index a48f8238cc5ca..91e160985bfd8 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -49,7 +49,6 @@ go_library( "//sessionctx/variable", "//sessiontxn", "//sessiontxn/isolation", - "//sessiontxn/legacy", "//sessiontxn/staleread", "//statistics", "//statistics/handle", diff --git a/session/session.go b/session/session.go index d01c3e7e549d3..05b36262e147e 100644 --- a/session/session.go +++ b/session/session.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/sessionstates" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/legacy" "github.com/pingcap/tidb/sessiontxn/staleread" "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/helper" @@ -2388,7 +2387,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, errors.Errorf("invalid CachedPrepareStmt type") } - var is infoschema.InfoSchema var snapshotTS uint64 replicaReadScope := oracle.GlobalTxnScope @@ -2400,7 +2398,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ txnManager := sessiontxn.GetTxnManager(s) if staleReadProcessor.IsStaleness() { snapshotTS = staleReadProcessor.GetStalenessReadTS() - is = staleReadProcessor.GetStalenessInfoSchema() + is := staleReadProcessor.GetStalenessInfoSchema() replicaReadScope = config.GetTxnScopeFromConfig() err = txnManager.EnterNewTxn(ctx, &sessiontxn.EnterNewTxnRequest{ Type: sessiontxn.EnterNewTxnWithReplaceProvider, @@ -2410,8 +2408,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ if err != nil { return nil, err } - } else { - is = s.GetInfoSchema().(infoschema.InfoSchema) } staleness := snapshotTS > 0 @@ -2427,10 +2423,6 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return nil, err } - if p, isOK := txnManager.GetContextProvider().(*legacy.SimpleTxnContextProvider); isOK { - p.InfoSchema = is - } - if ok { rs, ok, err := s.cachedPointPlanExec(ctx, txnManager.GetTxnInfoSchema(), stmtID, preparedStmt, replicaReadScope, args) if err != nil { diff --git a/sessionctx/sessionstates/BUILD.bazel b/sessionctx/sessionstates/BUILD.bazel index ba5cb9254f9f8..0b0d2c6d1c6e8 100644 --- a/sessionctx/sessionstates/BUILD.bazel +++ b/sessionctx/sessionstates/BUILD.bazel @@ -7,6 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//parser/types", + "//sessionctx/stmtctx", "//types", ], ) diff --git a/sessionctx/stmtctx/BUILD.bazel b/sessionctx/stmtctx/BUILD.bazel index 168b86f32cf62..c59196a17766c 100644 --- a/sessionctx/stmtctx/BUILD.bazel +++ b/sessionctx/stmtctx/BUILD.bazel @@ -10,12 +10,14 @@ go_library( "//parser/ast", "//parser/model", "//parser/mysql", + "//parser/terror", "//util/disk", "//util/execdetails", "//util/memory", "//util/resourcegrouptag", "//util/topsql/stmtstats", "//util/tracing", + "@com_github_pingcap_errors//:errors", "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_tikv_client_go_v2//util", "@org_uber_go_atomic//:atomic", @@ -32,9 +34,11 @@ go_test( embed = [":stmtctx"], deps = [ "//kv", + "//sessionctx/variable", "//testkit", "//testkit/testsetup", "//util/execdetails", + "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//util", "@org_uber_go_goleak//:goleak", diff --git a/sessiontxn/isolation/BUILD.bazel b/sessiontxn/isolation/BUILD.bazel index 35fb911f4b636..1345c09f08b21 100644 --- a/sessiontxn/isolation/BUILD.bazel +++ b/sessiontxn/isolation/BUILD.bazel @@ -53,6 +53,7 @@ go_test( "//testkit", "//testkit/testsetup", "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/kvrpcpb", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//error", diff --git a/sessiontxn/legacy/BUILD.bazel b/sessiontxn/legacy/BUILD.bazel deleted file mode 100644 index d9e9b5249cbbf..0000000000000 --- a/sessiontxn/legacy/BUILD.bazel +++ /dev/null @@ -1,41 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "legacy", - srcs = ["provider.go"], - importpath = "github.com/pingcap/tidb/sessiontxn/legacy", - visibility = ["//visibility:public"], - deps = [ - "//domain", - "//infoschema", - "//kv", - "//parser/ast", - "//parser/terror", - "//sessionctx", - "//sessionctx/variable", - "//sessiontxn", - "//sessiontxn/staleread", - "//table/temptable", - "//util/logutil", - "@com_github_pingcap_errors//:errors", - "@com_github_tikv_client_go_v2//error", - "@org_uber_go_zap//:zap", - ], -) - -go_test( - name = "legacy_test", - srcs = ["provider_test.go"], - deps = [ - ":legacy", - "//domain", - "//kv", - "//sessionctx", - "//sessiontxn", - "//testkit", - "@com_github_pingcap_errors//:errors", - "@com_github_pingcap_kvproto//pkg/kvrpcpb", - "@com_github_stretchr_testify//require", - "@com_github_tikv_client_go_v2//error", - ], -) diff --git a/sessiontxn/legacy/provider.go b/sessiontxn/legacy/provider.go deleted file mode 100644 index a571c612f24aa..0000000000000 --- a/sessiontxn/legacy/provider.go +++ /dev/null @@ -1,243 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package legacy - -import ( - "context" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/staleread" - "github.com/pingcap/tidb/table/temptable" - "github.com/pingcap/tidb/util/logutil" - tikverr "github.com/tikv/client-go/v2/error" - "go.uber.org/zap" -) - -// SimpleTxnContextProvider implements TxnContextProvider -// It is only used in refactor stage -// TODO: remove it after refactor finished -type SimpleTxnContextProvider struct { - Ctx context.Context - Sctx sessionctx.Context - InfoSchema infoschema.InfoSchema - GetReadTSFunc func() (uint64, error) - GetForUpdateTSFunc func() (uint64, error) - UpdateForUpdateTS func(seCtx sessionctx.Context, newForUpdateTS uint64) error - - Pessimistic bool - CausalConsistencyOnly bool - - isTxnActive bool -} - -// GetTxnInfoSchema returns the information schema used by txn -func (p *SimpleTxnContextProvider) GetTxnInfoSchema() infoschema.InfoSchema { - return p.InfoSchema -} - -// GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update) -func (p *SimpleTxnContextProvider) GetStmtReadTS() (uint64, error) { - if p.GetReadTSFunc == nil { - return 0, errors.New("ReadTSFunc not set") - } - return p.GetReadTSFunc() -} - -// GetStmtForUpdateTS returns the read timestamp used by update/insert/delete or select ... for update -func (p *SimpleTxnContextProvider) GetStmtForUpdateTS() (uint64, error) { - if p.GetForUpdateTSFunc == nil { - return 0, errors.New("GetForUpdateTSFunc not set") - } - return p.GetForUpdateTSFunc() -} - -// OnInitialize is the hook that should be called when enter a new txn with this provider -func (p *SimpleTxnContextProvider) OnInitialize(ctx context.Context, tp sessiontxn.EnterNewTxnType) error { - p.Ctx = ctx - sessVars := p.Sctx.GetSessionVars() - switch tp { - case sessiontxn.EnterNewTxnDefault, sessiontxn.EnterNewTxnWithBeginStmt: - shouldReuseTxn := tp == sessiontxn.EnterNewTxnWithBeginStmt && sessiontxn.CanReuseTxnWhenExplicitBegin(p.Sctx) - if !shouldReuseTxn { - if err := p.Sctx.NewTxn(ctx); err != nil { - return err - } - } - - if tp == sessiontxn.EnterNewTxnWithBeginStmt { - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - sessVars.SetInTxn(true) - } - - sessVars.TxnCtx.IsPessimistic = p.Pessimistic - if _, err := p.activateTxn(); err != nil { - return err - } - - if is, ok := sessVars.TxnCtx.InfoSchema.(infoschema.InfoSchema); ok { - p.InfoSchema = is - } - case sessiontxn.EnterNewTxnBeforeStmt: - p.InfoSchema = temptable.AttachLocalTemporaryTableInfoSchema(p.Sctx, domain.GetDomain(p.Sctx).InfoSchema()) - sessVars.TxnCtx = &variable.TransactionContext{ - TxnCtxNoNeedToRestore: variable.TxnCtxNoNeedToRestore{ - InfoSchema: p.InfoSchema, - CreateTime: time.Now(), - ShardStep: int(sessVars.ShardAllocateStep), - TxnScope: sessVars.CheckAndGetTxnScope(), - IsPessimistic: p.Pessimistic, - }, - } - default: - return errors.Errorf("Unsupported type: %v", tp) - } - - return nil -} - -// OnStmtStart is the hook that should be called when a new statement started -func (p *SimpleTxnContextProvider) OnStmtStart(ctx context.Context) error { - p.Ctx = ctx - p.InfoSchema = p.Sctx.GetInfoSchema().(infoschema.InfoSchema) - return nil -} - -// OnStmtErrorForNextAction is the hook that should be called when a new statement get an error -func (p *SimpleTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { - switch point { - case sessiontxn.StmtErrAfterPessimisticLock: - return p.handleAfterPessimisticLockError(err) - default: - return sessiontxn.NoIdea() - } -} - -func (p *SimpleTxnContextProvider) handleAfterPessimisticLockError(lockErr error) (sessiontxn.StmtErrorAction, error) { - sessVars := p.Sctx.GetSessionVars() - if sessVars.IsIsolation(ast.Serializable) { - return sessiontxn.ErrorAction(lockErr) - } - - txnCtx := sessVars.TxnCtx - if deadlock, ok := errors.Cause(lockErr).(*tikverr.ErrDeadlock); ok { - if !deadlock.IsRetryable { - return sessiontxn.ErrorAction(lockErr) - } - logutil.Logger(p.Ctx).Info("single statement deadlock, retry statement", - zap.Uint64("txn", txnCtx.StartTS), - zap.Uint64("lockTS", deadlock.LockTs), - zap.Stringer("lockKey", kv.Key(deadlock.LockKey)), - zap.Uint64("deadlockKeyHash", deadlock.DeadlockKeyHash)) - } else if terror.ErrorEqual(kv.ErrWriteConflict, lockErr) { - errStr := lockErr.Error() - forUpdateTS := txnCtx.GetForUpdateTS() - logutil.Logger(p.Ctx).Debug("pessimistic write conflict, retry statement", - zap.Uint64("txn", txnCtx.StartTS), - zap.Uint64("forUpdateTS", forUpdateTS), - zap.String("err", errStr)) - // Always update forUpdateTS by getting a new timestamp from PD. - // If we use the conflict commitTS as the new forUpdateTS and async commit - // is used, the commitTS of this transaction may exceed the max timestamp - // that PD allocates. Then, the change may be invisible to a new transaction, - // which means linearizability is broken. - } else { - // this branch if err not nil, always update forUpdateTS to avoid problem described below - // for nowait, when ErrLock happened, ErrLockAcquireFailAndNoWaitSet will be returned, and in the same txn - // the select for updateTs must be updated, otherwise there maybe rollback problem. - // begin; select for update key1(here ErrLocked or other errors(or max_execution_time like util), - // key1 lock not get and async rollback key1 is raised) - // select for update key1 again(this time lock succ(maybe lock released by others)) - // the async rollback operation rollbacked the lock just acquired - tsErr := p.UpdateForUpdateTS(p.Sctx, 0) - if tsErr != nil { - logutil.Logger(p.Ctx).Warn("UpdateForUpdateTS failed", zap.Error(tsErr)) - } - return sessiontxn.ErrorAction(lockErr) - } - - if err := p.UpdateForUpdateTS(p.Sctx, 0); err != nil { - return sessiontxn.ErrorAction(lockErr) - } - - return sessiontxn.RetryReady() -} - -// OnStmtRetry is the hook that should be called when a statement retry -func (p *SimpleTxnContextProvider) OnStmtRetry(_ context.Context) error { - return nil -} - -func (p *SimpleTxnContextProvider) prepareTSFuture() error { - if p.Sctx.GetSessionVars().SnapshotTS != 0 || staleread.IsStmtStaleness(p.Sctx) || p.Sctx.GetPreparedTSFuture() != nil { - return nil - } - - txn, err := p.Sctx.Txn(false) - if err != nil { - return err - } - - if txn.Valid() { - return nil - } - - txnScope := p.Sctx.GetSessionVars().CheckAndGetTxnScope() - future := sessiontxn.NewOracleFuture(p.Ctx, p.Sctx, txnScope) - return p.Sctx.PrepareTSFuture(p.Ctx, future, txnScope) -} - -// activateTxn actives the txn -func (p *SimpleTxnContextProvider) activateTxn() (kv.Transaction, error) { - if p.isTxnActive { - return p.Sctx.Txn(true) - } - - txn, err := p.Sctx.Txn(true) - if err != nil { - return nil, err - } - - if p.Pessimistic { - txn.SetOption(kv.Pessimistic, true) - } - - if p.CausalConsistencyOnly { - txn.SetOption(kv.GuaranteeLinearizability, false) - } - - p.isTxnActive = true - return txn, nil -} - -// AdviseWarmup provides warmup for inner state -func (p *SimpleTxnContextProvider) AdviseWarmup() error { - return p.prepareTSFuture() -} - -// AdviseOptimizeWithPlan providers optimization according to the plan -func (p *SimpleTxnContextProvider) AdviseOptimizeWithPlan(_ interface{}) error { - return nil -} diff --git a/sessiontxn/legacy/provider_test.go b/sessiontxn/legacy/provider_test.go deleted file mode 100644 index 22aa042632182..0000000000000 --- a/sessiontxn/legacy/provider_test.go +++ /dev/null @@ -1,126 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package legacy_test - -import ( - "context" - "testing" - - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/sessiontxn/legacy" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" - tikverr "github.com/tikv/client-go/v2/error" -) - -func TestErrorHandle(t *testing.T) { - store, do, clean := testkit.CreateMockStoreAndDomain(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - - provider := newSimpleProvider(tk, do) - require.NoError(t, provider.OnStmtStart(context.TODO())) - expectedForUpdateTS := getForUpdateTS(t, provider) - - var lockErr error - - // StmtErrAfterLock: ErrWriteConflict should retry and update forUpdateTS - lockErr = kv.ErrWriteConflict - action, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionRetryReady, action) - require.Nil(t, err) - expectedForUpdateTS += 1 - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterLock: DeadLock that is not retryable will just return an error - lockErr = newDeadLockError(false) - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionError, action) - require.Equal(t, lockErr, err) - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterLock: DeadLock that is retryable should retry and update forUpdateTS - lockErr = newDeadLockError(true) - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionRetryReady, action) - require.Nil(t, err) - expectedForUpdateTS += 1 - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterLock: other errors should only update forUpdateTS but not retry - lockErr = errors.New("other error") - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) - require.Equal(t, sessiontxn.StmtActionError, action) - require.Equal(t, lockErr, err) - expectedForUpdateTS += 1 - require.Equal(t, expectedForUpdateTS, getForUpdateTS(t, provider)) - - // StmtErrAfterQuery: always not retry - lockErr = kv.ErrWriteConflict - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, lockErr) - require.Equal(t, sessiontxn.StmtActionNoIdea, action) - require.Nil(t, err) - - tk.Session().GetSessionVars().StmtCtx.RCCheckTS = true - require.NoError(t, provider.OnStmtStart(context.TODO())) - action, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterQuery, lockErr) - require.Equal(t, sessiontxn.StmtActionNoIdea, action) - require.Nil(t, err) -} - -func getForUpdateTS(t *testing.T, provider *legacy.SimpleTxnContextProvider) uint64 { - forUpdateTS, err := provider.GetStmtForUpdateTS() - require.NoError(t, err) - return forUpdateTS -} - -func newDeadLockError(isRetryable bool) error { - return &tikverr.ErrDeadlock{ - Deadlock: &kvrpcpb.Deadlock{}, - IsRetryable: isRetryable, - } -} - -func newSimpleProvider(tk *testkit.TestKit, do *domain.Domain) *legacy.SimpleTxnContextProvider { - tk.MustExec("begin pessimistic") - readTS := uint64(1) - forUpdateTS := uint64(1) - return &legacy.SimpleTxnContextProvider{ - Ctx: context.TODO(), - Sctx: tk.Session(), - InfoSchema: do.InfoSchema(), - GetReadTSFunc: func() (uint64, error) { - return readTS, nil - }, - GetForUpdateTSFunc: func() (uint64, error) { - return forUpdateTS, nil - }, - UpdateForUpdateTS: func(seCtx sessionctx.Context, newForUpdateTS uint64) error { - if newForUpdateTS == 0 { - forUpdateTS += 1 - } else { - forUpdateTS = newForUpdateTS - } - return nil - }, - Pessimistic: true, - } -} diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 9fec1844606a3..01cb607e68a33 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -52,7 +52,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerAfterPessimisticLockErrorRetry", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder", "return")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec", "return")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) @@ -83,7 +82,6 @@ func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerAfterPessimisticLockErrorRetry")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleReadValuesSameWithExecuteAndBuilder")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertNotStaleReadForExecutorGetReadTS")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) From 9bde478b22e2f554f3df6eb917fc7bb6fdd82dfd Mon Sep 17 00:00:00 2001 From: likzn <1020193211@qq.com> Date: Wed, 29 Jun 2022 15:48:39 +0800 Subject: [PATCH 23/28] ddl: make output field name in `show tables/databases stmt` compatible with mysql (#35136) close pingcap/tidb#35116 --- cmd/explaintest/r/show.result | 4 ++++ cmd/explaintest/t/show.test | 3 +++ planner/core/planbuilder.go | 34 +++++++++++++++++++++++++++------- 3 files changed, 34 insertions(+), 7 deletions(-) create mode 100644 cmd/explaintest/r/show.result create mode 100644 cmd/explaintest/t/show.test diff --git a/cmd/explaintest/r/show.result b/cmd/explaintest/r/show.result new file mode 100644 index 0000000000000..6dfbf77c4af44 --- /dev/null +++ b/cmd/explaintest/r/show.result @@ -0,0 +1,4 @@ +show tables like '%xx'; +Tables_in_test (%xx) +show databases like '%xx'; +Database (%xx) diff --git a/cmd/explaintest/t/show.test b/cmd/explaintest/t/show.test new file mode 100644 index 0000000000000..b90131d18f861 --- /dev/null +++ b/cmd/explaintest/t/show.test @@ -0,0 +1,3 @@ +# test show output field name +show tables like '%xx'; +show databases like '%xx'; diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b150788d36a56..1059e8eadf1a5 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2933,17 +2933,17 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, }.Init(b.ctx) isView := false isSequence := false + // It depends on ShowPredicateExtractor now + buildPattern := true switch show.Tp { case ast.ShowDatabases, ast.ShowVariables, ast.ShowTables, ast.ShowColumns, ast.ShowTableStatus, ast.ShowCollation: if (show.Tp == ast.ShowTables || show.Tp == ast.ShowTableStatus) && p.DBName == "" { return nil, ErrNoDB } - extractor := newShowBaseExtractor(*show) - if extractor.Extract() { + if extractor := newShowBaseExtractor(*show); extractor.Extract() { p.Extractor = extractor - // Avoid building Selection. - show.Pattern = nil + buildPattern = false } case ast.ShowCreateTable, ast.ShowCreateSequence, ast.ShowPlacementForTable, ast.ShowPlacementForPartition: var err error @@ -3019,7 +3019,8 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, var err error var np LogicalPlan np = p - if show.Pattern != nil { + // If we have ShowPredicateExtractor, we do not buildSelection with Pattern + if show.Pattern != nil && buildPattern { show.Pattern.Expr = &ast.ColumnNameExpr{ Name: &ast.ColumnName{Name: p.OutputNames()[0].ColName}, } @@ -4645,12 +4646,20 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp case ast.ShowConfig: names = []string{"Type", "Instance", "Name", "Value"} case ast.ShowDatabases: - names = []string{"Database"} + fieldDB := "Database" + if patternName := extractPatternLikeName(s.Pattern); patternName != "" { + fieldDB = fmt.Sprintf("%s (%s)", fieldDB, patternName) + } + names = []string{fieldDB} case ast.ShowOpenTables: names = []string{"Database", "Table", "In_use", "Name_locked"} ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeLong} case ast.ShowTables: - names = []string{fmt.Sprintf("Tables_in_%s", s.DBName)} + fieldTable := fmt.Sprintf("Tables_in_%s", s.DBName) + if patternName := extractPatternLikeName(s.Pattern); patternName != "" { + fieldTable = fmt.Sprintf("%s (%s)", fieldTable, patternName) + } + names = []string{fieldTable} if s.Full { names = append(names, "Table_type") } @@ -4870,3 +4879,14 @@ func (b *PlanBuilder) buildCompactTable(node *ast.CompactTableStmt) (Plan, error } return p, nil } + +func extractPatternLikeName(patternLike *ast.PatternLikeExpr) string { + if patternLike == nil { + return "" + } + switch v := patternLike.Pattern.(type) { + case *driver.ValueExpr: + return v.GetString() + } + return "" +} From 5b305c34f80584f92efd750fe29b1e3241e9fc1d Mon Sep 17 00:00:00 2001 From: Spade A <71589810+SpadeA-Tang@users.noreply.github.com> Date: Wed, 29 Jun 2022 16:14:39 +0800 Subject: [PATCH 24/28] txn: manage the initialization of RCCheckTS by transaction context provider (#35554) close pingcap/tidb#35497 --- executor/adapter.go | 5 + executor/executor.go | 15 --- executor/executor_test.go | 2 +- executor/prepared.go | 8 +- executor/seqtest/prepared_test.go | 4 +- go.mod | 2 +- planner/core/preprocess.go | 2 +- planner/funcdep/extract_fd_test.go | 4 +- session/bench_test.go | 4 +- session/session.go | 33 +++---- session/txnmanager.go | 12 ++- sessiontxn/interface.go | 9 +- sessiontxn/isolation/base.go | 3 +- sessiontxn/isolation/optimistic_test.go | 28 +++--- sessiontxn/isolation/readcommitted.go | 21 ++++- sessiontxn/isolation/readcommitted_test.go | 94 +++++++++++++++---- sessiontxn/isolation/repeatable_read.go | 4 +- sessiontxn/isolation/repeatable_read_test.go | 30 +++--- sessiontxn/isolation/serializable_test.go | 18 ++-- sessiontxn/staleread/provider.go | 3 +- sessiontxn/txn_manager_test.go | 2 +- .../sessiontest/temporary_table_test.go | 2 +- util/sqlexec/restricted_sql_executor.go | 3 + 23 files changed, 194 insertions(+), 114 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 31a8c8d20150f..3a4ace6ba582b 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -223,6 +223,11 @@ type ExecStmt struct { Ti *TelemetryInfo } +// GetStmtNode returns the stmtNode inside Statement +func (a ExecStmt) GetStmtNode() ast.StmtNode { + return a.StmtNode +} + // PointGet short path for point exec directly from plan, keep only necessary steps func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*recordSet, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { diff --git a/executor/executor.go b/executor/executor.go index 9bb2cd5789d65..7b370615582cc 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1922,11 +1922,6 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.NotFillCache = !opts.SQLCache } sc.WeakConsistency = isWeakConsistencyRead(ctx, stmt) - // Try to mark the `RCCheckTS` flag for the first time execution of in-transaction read requests - // using read-consistency isolation level. - if NeedSetRCCheckTSFlag(ctx, stmt) { - sc.RCCheckTS = true - } case *ast.SetOprStmt: sc.InSelectStmt = true sc.OverflowAsWarning = true @@ -2065,13 +2060,3 @@ func isWeakConsistencyRead(ctx sessionctx.Context, node ast.Node) bool { return sessionVars.ConnectionID > 0 && sessionVars.ReadConsistency.IsWeak() && plannercore.IsAutoCommitTxn(ctx) && plannercore.IsReadOnly(node, sessionVars) } - -// NeedSetRCCheckTSFlag checks whether it's needed to set `RCCheckTS` flag in current stmtctx. -func NeedSetRCCheckTSFlag(ctx sessionctx.Context, node ast.Node) bool { - sessionVars := ctx.GetSessionVars() - if sessionVars.ConnectionID > 0 && sessionVars.RcReadCheckTS && sessionVars.InTxn() && - sessionVars.IsPessimisticReadConsistency() && !sessionVars.RetryInfo.Retrying && plannercore.IsReadOnly(node, sessionVars) { - return true - } - return false -} diff --git a/executor/executor_test.go b/executor/executor_test.go index c8e4304f1c22c..fb0b9816ccbf7 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3470,7 +3470,7 @@ func TestUnreasonablyClose(t *testing.T) { err = sessiontxn.NewTxn(context.Background(), tk.Session()) require.NoError(t, err, comment) - err = sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO()) + err = sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO(), stmt) require.NoError(t, err, comment) executorBuilder := executor.NewMockExecutorBuilderForTest(tk.Session(), is, nil, oracle.GlobalTxnScope) diff --git a/executor/prepared.go b/executor/prepared.go index e6395371c95d5..abe974e899310 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -333,15 +333,11 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { // CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement. func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, - ID uint32, is infoschema.InfoSchema, snapshotTS uint64, replicaReadScope string, args []types.Datum) (*ExecStmt, bool, bool, error) { + execStmt *ast.ExecuteStmt, is infoschema.InfoSchema, snapshotTS uint64, replicaReadScope string, args []types.Datum) (*ExecStmt, bool, bool, error) { startTime := time.Now() defer func() { sctx.GetSessionVars().DurationCompile = time.Since(startTime) }() - execStmt := &ast.ExecuteStmt{ExecID: ID} - if err := ResetContextOfStmt(sctx, execStmt); err != nil { - return nil, false, false, err - } isStaleness := snapshotTS != 0 sctx.GetSessionVars().StmtCtx.IsStaleness = isStaleness execStmt.BinaryArgs = args @@ -369,7 +365,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, Ti: &TelemetryInfo{}, ReplicaReadScope: replicaReadScope, } - if preparedPointer, ok := sctx.GetSessionVars().PreparedStmts[ID]; ok { + if preparedPointer, ok := sctx.GetSessionVars().PreparedStmts[execStmt.ExecID]; ok { preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) if !ok { return nil, false, false, errors.Errorf("invalid CachedPrepareStmt type") diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index b39f66d3030ee..b4b018e8ac3ee 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" @@ -157,8 +158,9 @@ func TestPrepared(t *testing.T) { require.NoError(t, err) tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows()) + execStmt := &ast.ExecuteStmt{ExecID: stmtID} // Check that ast.Statement created by executor.CompileExecutePreparedStmt has query text. - stmt, _, _, err := executor.CompileExecutePreparedStmt(context.TODO(), tk.Session(), stmtID, + stmt, _, _, err := executor.CompileExecutePreparedStmt(context.TODO(), tk.Session(), execStmt, tk.Session().GetInfoSchema().(infoschema.InfoSchema), 0, kv.GlobalReplicaScope, []types.Datum{types.NewDatum(1)}) require.NoError(t, err) require.Equal(t, query, stmt.OriginText()) diff --git a/go.mod b/go.mod index 9ad9135be4085..8943a2403025a 100644 --- a/go.mod +++ b/go.mod @@ -117,7 +117,7 @@ require ( github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/cespare/xxhash/v2 v2.1.2 // indirect - github.com/cockroachdb/errors v1.8.1 // indirect + github.com/cockroachdb/errors v1.8.1 github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f // indirect github.com/cockroachdb/redact v1.0.8 // indirect github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 // indirect diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 3d55d3f6c14d2..df47167354c2a 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -1653,7 +1653,7 @@ func (p *preprocessor) updateStateFromStaleReadProcessor() error { if err := txnManager.EnterNewTxn(context.TODO(), newTxnRequest); err != nil { return err } - if err := txnManager.OnStmtStart(context.TODO()); err != nil { + if err := txnManager.OnStmtStart(context.TODO(), txnManager.GetCurrentStmt()); err != nil { return err } } diff --git a/planner/funcdep/extract_fd_test.go b/planner/funcdep/extract_fd_test.go index b5bb646cba073..aed58f6dd2957 100644 --- a/planner/funcdep/extract_fd_test.go +++ b/planner/funcdep/extract_fd_test.go @@ -214,7 +214,7 @@ func TestFDSet_ExtractFD(t *testing.T) { for i, tt := range tests { comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO())) - require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO())) + require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO(), nil)) stmt, err := par.ParseOneStmt(tt.sql, "", "") require.NoError(t, err, comment) tk.Session().GetSessionVars().PlanID = 0 @@ -312,7 +312,7 @@ func TestFDSet_ExtractFDForApply(t *testing.T) { is := testGetIS(t, tk.Session()) for i, tt := range tests { require.NoError(t, tk.Session().PrepareTxnCtx(context.TODO())) - require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO())) + require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO(), nil)) comment := fmt.Sprintf("case:%v sql:%s", i, tt.sql) stmt, err := par.ParseOneStmt(tt.sql, "", "") require.NoError(t, err, comment) diff --git a/session/bench_test.go b/session/bench_test.go index c1164ec32de40..75be9443cf7e6 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/benchdaily" @@ -1812,8 +1813,9 @@ func BenchmarkCompileExecutePreparedStmt(b *testing.B) { is := se.GetInfoSchema() b.ResetTimer() + stmtExec := &ast.ExecuteStmt{ExecID: stmtID} for i := 0; i < b.N; i++ { - _, _, _, err := executor.CompileExecutePreparedStmt(context.Background(), se, stmtID, is.(infoschema.InfoSchema), 0, kv.GlobalTxnScope, args) + _, _, _, err := executor.CompileExecutePreparedStmt(context.Background(), se, stmtExec, is.(infoschema.InfoSchema), 0, kv.GlobalTxnScope, args) if err != nil { b.Fatal(err) } diff --git a/session/session.go b/session/session.go index 05b36262e147e..5d6a50246bce1 100644 --- a/session/session.go +++ b/session/session.go @@ -1111,7 +1111,7 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) { } _, digest := s.sessionVars.StmtCtx.SQLDigest() s.txn.onStmtStart(digest.String()) - if err = sessiontxn.GetTxnManager(s).OnStmtStart(ctx); err == nil { + if err = sessiontxn.GetTxnManager(s).OnStmtStart(ctx, st.GetStmtNode()); err == nil { _, err = st.Exec(ctx) } s.txn.onStmtEnd() @@ -1917,7 +1917,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex s.txn.onStmtStart(digest.String()) defer s.txn.onStmtEnd() - if err := s.onTxnManagerStmtStartOrRetry(ctx); err != nil { + if err := s.onTxnManagerStmtStartOrRetry(ctx, stmtNode); err != nil { return nil, err } @@ -1980,11 +1980,11 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex return recordSet, nil } -func (s *session) onTxnManagerStmtStartOrRetry(ctx context.Context) error { +func (s *session) onTxnManagerStmtStartOrRetry(ctx context.Context, node ast.StmtNode) error { if s.sessionVars.RetryInfo.Retrying { return sessiontxn.GetTxnManager(s).OnStmtRetry(ctx) } - return sessiontxn.GetTxnManager(s).OnStmtStart(ctx) + return sessiontxn.GetTxnManager(s).OnStmtStart(ctx, node) } func (s *session) validateStatementReadOnlyInStaleness(stmtNode ast.StmtNode) error { @@ -2201,7 +2201,8 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields return } - if err = s.onTxnManagerStmtStartOrRetry(ctx); err != nil { + prepareStmt := &ast.PrepareStmt{SQLText: sql} + if err = s.onTxnManagerStmtStartOrRetry(ctx, prepareStmt); err != nil { return } @@ -2222,7 +2223,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields func (s *session) preparedStmtExec(ctx context.Context, is infoschema.InfoSchema, snapshotTS uint64, - stmtID uint32, prepareStmt *plannercore.CachedPrepareStmt, replicaReadScope string, args []types.Datum) (sqlexec.RecordSet, error) { + execStmt *ast.ExecuteStmt, prepareStmt *plannercore.CachedPrepareStmt, replicaReadScope string, args []types.Datum) (sqlexec.RecordSet, error) { failpoint.Inject("assertTxnManagerInPreparedStmtExec", func() { sessiontxn.RecordAssert(s, "assertTxnManagerInPreparedStmtExec", true) @@ -2232,7 +2233,7 @@ func (s *session) preparedStmtExec(ctx context.Context, } }) - st, tiFlashPushDown, tiFlashExchangePushDown, err := executor.CompileExecutePreparedStmt(ctx, s, stmtID, is, snapshotTS, replicaReadScope, args) + st, tiFlashPushDown, tiFlashExchangePushDown, err := executor.CompileExecutePreparedStmt(ctx, s, execStmt, is, snapshotTS, replicaReadScope, args) if err != nil { return nil, err } @@ -2252,14 +2253,9 @@ func (s *session) preparedStmtExec(ctx context.Context, // cachedPointPlanExec is a short path currently ONLY for cached "point select plan" execution func (s *session) cachedPointPlanExec(ctx context.Context, - is infoschema.InfoSchema, stmtID uint32, prepareStmt *plannercore.CachedPrepareStmt, replicaReadScope string, args []types.Datum) (sqlexec.RecordSet, bool, error) { + is infoschema.InfoSchema, execAst *ast.ExecuteStmt, prepareStmt *plannercore.CachedPrepareStmt, replicaReadScope string, args []types.Datum) (sqlexec.RecordSet, bool, error) { prepared := prepareStmt.PreparedAst - // compile ExecStmt - execAst := &ast.ExecuteStmt{ExecID: stmtID} - if err := executor.ResetContextOfStmt(s, execAst); err != nil { - return nil, false, err - } failpoint.Inject("assertTxnManagerInCachedPlanExec", func() { sessiontxn.RecordAssert(s, "assertTxnManagerInCachedPlanExec", true) @@ -2419,12 +2415,17 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ s.txn.onStmtStart(preparedStmt.SQLDigest.String()) defer s.txn.onStmtEnd() - if err = s.onTxnManagerStmtStartOrRetry(ctx); err != nil { + execStmt := &ast.ExecuteStmt{ExecID: stmtID} + if err := executor.ResetContextOfStmt(s, execStmt); err != nil { + return nil, err + } + + if err = s.onTxnManagerStmtStartOrRetry(ctx, execStmt); err != nil { return nil, err } if ok { - rs, ok, err := s.cachedPointPlanExec(ctx, txnManager.GetTxnInfoSchema(), stmtID, preparedStmt, replicaReadScope, args) + rs, ok, err := s.cachedPointPlanExec(ctx, txnManager.GetTxnInfoSchema(), execStmt, preparedStmt, replicaReadScope, args) if err != nil { return nil, err } @@ -2432,7 +2433,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ return rs, nil } } - return s.preparedStmtExec(ctx, txnManager.GetTxnInfoSchema(), snapshotTS, stmtID, preparedStmt, replicaReadScope, args) + return s.preparedStmtExec(ctx, txnManager.GetTxnInfoSchema(), snapshotTS, execStmt, preparedStmt, replicaReadScope, args) } func (s *session) DropPreparedStmt(stmtID uint32) error { diff --git a/session/txnmanager.go b/session/txnmanager.go index 19d28ae014f70..3d5a049307eb9 100644 --- a/session/txnmanager.go +++ b/session/txnmanager.go @@ -46,6 +46,7 @@ func getTxnManager(sctx sessionctx.Context) sessiontxn.TxnManager { type txnManager struct { sctx sessionctx.Context + stmtNode ast.StmtNode ctxProvider sessiontxn.TxnContextProvider } @@ -116,14 +117,21 @@ func (m *txnManager) EnterNewTxn(ctx context.Context, r *sessiontxn.EnterNewTxnR func (m *txnManager) OnTxnEnd() { m.ctxProvider = nil + m.stmtNode = nil +} + +func (m *txnManager) GetCurrentStmt() ast.StmtNode { + return m.stmtNode } // OnStmtStart is the hook that should be called when a new statement started -func (m *txnManager) OnStmtStart(ctx context.Context) error { +func (m *txnManager) OnStmtStart(ctx context.Context, node ast.StmtNode) error { + m.stmtNode = node + if m.ctxProvider == nil { return errors.New("context provider not set") } - return m.ctxProvider.OnStmtStart(ctx) + return m.ctxProvider.OnStmtStart(ctx, m.stmtNode) } // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error diff --git a/sessiontxn/interface.go b/sessiontxn/interface.go index d1febc88c8a48..ad41877e7439d 100644 --- a/sessiontxn/interface.go +++ b/sessiontxn/interface.go @@ -124,7 +124,7 @@ type TxnContextProvider interface { // OnInitialize is the hook that should be called when enter a new txn with this provider OnInitialize(ctx context.Context, enterNewTxnType EnterNewTxnType) error // OnStmtStart is the hook that should be called when a new statement started - OnStmtStart(ctx context.Context) error + OnStmtStart(ctx context.Context, node ast.StmtNode) error // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error OnStmtErrorForNextAction(point StmtErrorHandlePoint, err error) (StmtErrorAction, error) // OnStmtRetry is the hook that should be called when a statement is retried internally. @@ -150,7 +150,7 @@ type TxnManager interface { // OnTxnEnd is the hook that should be called after transaction commit or rollback OnTxnEnd() // OnStmtStart is the hook that should be called when a new statement started - OnStmtStart(ctx context.Context) error + OnStmtStart(ctx context.Context, node ast.StmtNode) error // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error // This method is not required to be called for every error in the statement, // it is only required to be called for some errors handled in some specified points given by the parameter `point`. @@ -158,6 +158,8 @@ type TxnManager interface { OnStmtErrorForNextAction(point StmtErrorHandlePoint, err error) (StmtErrorAction, error) // OnStmtRetry is the hook that should be called when a statement retry OnStmtRetry(ctx context.Context) error + // GetCurrentStmt returns the current statement node + GetCurrentStmt() ast.StmtNode } // NewTxn starts a new optimistic and active txn, it can be used for the below scenes: @@ -178,7 +180,8 @@ func NewTxnInStmt(ctx context.Context, sctx sessionctx.Context) error { if err := NewTxn(ctx, sctx); err != nil { return err } - return GetTxnManager(sctx).OnStmtStart(ctx) + txnManager := GetTxnManager(sctx) + return txnManager.OnStmtStart(ctx, txnManager.GetCurrentStmt()) } // GetTxnManager returns the TxnManager object from session context diff --git a/sessiontxn/isolation/base.go b/sessiontxn/isolation/base.go index d5c3bcbca7ab1..92e691d5a94df 100644 --- a/sessiontxn/isolation/base.go +++ b/sessiontxn/isolation/base.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -138,7 +139,7 @@ func (p *baseTxnContextProvider) GetStmtForUpdateTS() (uint64, error) { return p.getStmtForUpdateTSFunc() } -func (p *baseTxnContextProvider) OnStmtStart(ctx context.Context) error { +func (p *baseTxnContextProvider) OnStmtStart(ctx context.Context, _ ast.StmtNode) error { p.ctx = ctx return nil } diff --git a/sessiontxn/isolation/optimistic_test.go b/sessiontxn/isolation/optimistic_test.go index 25390dca1aa19..d85a53149a752 100644 --- a/sessiontxn/isolation/optimistic_test.go +++ b/sessiontxn/isolation/optimistic_test.go @@ -49,7 +49,7 @@ func TestOptimisticTxnContextProviderTS(t *testing.T) { se := tk.Session() compareTS := getOracleTS(t, se) provider := initializeOptimisticProvider(t, tk, true) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) readTS, err := provider.GetStmtReadTS() require.NoError(t, err) updateTS, err := provider.GetStmtForUpdateTS() @@ -59,7 +59,7 @@ func TestOptimisticTxnContextProviderTS(t *testing.T) { compareTS = readTS // for optimistic mode ts, ts should be the same for all statements - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) readTS, err = provider.GetStmtReadTS() require.NoError(t, err) updateTS, err = provider.GetStmtForUpdateTS() @@ -72,7 +72,7 @@ func TestOptimisticTxnContextProviderTS(t *testing.T) { require.NoError(t, err) stmt := stmts[0] provider = initializeOptimisticProvider(t, tk, false) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) plan, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, provider.GetTxnInfoSchema()) require.NoError(t, err) require.NoError(t, provider.AdviseOptimizeWithPlan(plan)) @@ -85,7 +85,7 @@ func TestOptimisticTxnContextProviderTS(t *testing.T) { // if the oracle future is prepared fist, `math.MaxUint64` should still be used after plan provider = initializeOptimisticProvider(t, tk, false) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) require.NoError(t, provider.AdviseWarmup()) plan, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, provider.GetTxnInfoSchema()) require.NoError(t, err) @@ -100,7 +100,7 @@ func TestOptimisticTxnContextProviderTS(t *testing.T) { // when it is in explicit txn, we should not use `math.MaxUint64` compareTS = getOracleTS(t, se) provider = initializeOptimisticProvider(t, tk, true) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) plan, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, provider.GetTxnInfoSchema()) require.NoError(t, err) require.NoError(t, provider.AdviseOptimizeWithPlan(plan)) @@ -115,7 +115,7 @@ func TestOptimisticTxnContextProviderTS(t *testing.T) { tk.MustExec("set @@autocommit=0") compareTS = getOracleTS(t, se) provider = initializeOptimisticProvider(t, tk, false) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) plan, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, provider.GetTxnInfoSchema()) require.NoError(t, err) require.NoError(t, provider.AdviseOptimizeWithPlan(plan)) @@ -175,7 +175,7 @@ func TestOptimisticHandleError(t *testing.T) { } for _, c := range cases { - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) action, err := provider.OnStmtErrorForNextAction(c.point, c.err) if c.point == sessiontxn.StmtErrAfterPessimisticLock { require.Error(t, err) @@ -183,7 +183,7 @@ func TestOptimisticHandleError(t *testing.T) { require.Equal(t, sessiontxn.StmtActionError, action) // next statement should not update ts - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkTS() } else { require.NoError(t, err) @@ -194,13 +194,13 @@ func TestOptimisticHandleError(t *testing.T) { checkTS() // OnStmtErrorForNextAction again - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) action, err = provider.OnStmtErrorForNextAction(c.point, c.err) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, action) // next statement should not update ts - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkTS() } } @@ -280,7 +280,7 @@ func TestOptimisticProviderInitialize(t *testing.T) { assertAfterActive.couldRetry = c.autocommit || !c.disableTxnAutoRetry require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider := assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -343,12 +343,12 @@ func TestTidbSnapshotVarInOptimisticTxn(t *testing.T) { } // information schema and ts should equal to snapshot when tidb_snapshot is set - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() // information schema and ts will restore when set tidb_snapshot to empty tk.MustExec("set @@tidb_snapshot=''") - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseTxn() // txn will not be active after `GetStmtReadTS` or `GetStmtForUpdateTS` when `tidb_snapshot` is set @@ -368,7 +368,7 @@ func TestTidbSnapshotVarInOptimisticTxn(t *testing.T) { assertAfterUseSnapshot := activeSnapshotTxnAssert(se, se.GetSessionVars().SnapshotTS, "") require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() assertAfterUseSnapshot.Check(t) }() diff --git a/sessiontxn/isolation/readcommitted.go b/sessiontxn/isolation/readcommitted.go index 985b6a04800f3..9918f8b226e5b 100644 --- a/sessiontxn/isolation/readcommitted.go +++ b/sessiontxn/isolation/readcommitted.go @@ -77,13 +77,30 @@ func NewPessimisticRCTxnContextProvider(sctx sessionctx.Context, causalConsisten } // OnStmtStart is the hook that should be called when a new statement started -func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context) error { - if err := p.baseTxnContextProvider.OnStmtStart(ctx); err != nil { +func (p *PessimisticRCTxnContextProvider) OnStmtStart(ctx context.Context, node ast.StmtNode) error { + if err := p.baseTxnContextProvider.OnStmtStart(ctx, node); err != nil { return err } + + // Try to mark the `RCCheckTS` flag for the first time execution of in-transaction read requests + // using read-consistency isolation level. + if node != nil && NeedSetRCCheckTSFlag(p.sctx, node) { + p.sctx.GetSessionVars().StmtCtx.RCCheckTS = true + } + return p.prepareStmt(!p.isTxnPrepared) } +// NeedSetRCCheckTSFlag checks whether it's needed to set `RCCheckTS` flag in current stmtctx. +func NeedSetRCCheckTSFlag(ctx sessionctx.Context, node ast.Node) bool { + sessionVars := ctx.GetSessionVars() + if sessionVars.ConnectionID > 0 && sessionVars.RcReadCheckTS && sessionVars.InTxn() && + !sessionVars.RetryInfo.Retrying && plannercore.IsReadOnly(node, sessionVars) { + return true + } + return false +} + // OnStmtErrorForNextAction is the hook that should be called when a new statement get an error func (p *PessimisticRCTxnContextProvider) OnStmtErrorForNextAction(point sessiontxn.StmtErrorHandlePoint, err error) (sessiontxn.StmtErrorAction, error) { switch point { diff --git a/sessiontxn/isolation/readcommitted_test.go b/sessiontxn/isolation/readcommitted_test.go index 5c747eba4fa2c..a01066f3588b1 100644 --- a/sessiontxn/isolation/readcommitted_test.go +++ b/sessiontxn/isolation/readcommitted_test.go @@ -29,10 +29,12 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/sessiontxn/isolation" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" "github.com/stretchr/testify/require" tikverr "github.com/tikv/client-go/v2/error" ) @@ -55,26 +57,26 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { forUpdateStmt := stmts[0] compareTS := se.GetSessionVars().TxnCtx.StartTS - // first ts should request from tso + // first ts should use the txn startTS require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, ts, compareTS) rcCheckTS := ts - // second ts should reuse first ts + // second ts should reuse the txn startTS require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, rcCheckTS, ts) // when one statement did not getStmtReadTS, the next one should still reuse the first ts require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, rcCheckTS, ts) @@ -93,7 +95,7 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { // if retry succeed next statement will still use rc check require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, rcCheckTS, ts) @@ -103,14 +105,14 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, rcCheckTS, ts) // `StmtErrAfterPessimisticLock` will still disable rc check require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), readOnlyStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, rcCheckTS, ts) @@ -128,7 +130,7 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { // only read-only stmt can retry for rc check require.NoError(t, executor.ResetContextOfStmt(se, forUpdateStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), forUpdateStmt)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) require.Greater(t, ts, compareTS) @@ -137,6 +139,60 @@ func TestPessimisticRCTxnContextProviderRCCheck(t *testing.T) { require.Equal(t, sessiontxn.StmtActionNoIdea, nextAction) } +func TestPessimisticRCTxnContextProviderRCCheckForPrepareExecute(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("create table t (id int primary key, v int)") + tk2.MustExec("insert into t values(1, 1)") + + tk.MustExec("set @@tidb_rc_read_check_ts=1") + se := tk.Session() + ctx := context.Background() + provider := initializePessimisticRCProvider(t, tk) + txnStartTS := se.GetSessionVars().TxnCtx.StartTS + + // first ts should use the txn startTS + stmt, _, _, err := tk.Session().PrepareStmt("select * from t") + require.NoError(t, err) + rs, err := tk.Session().ExecutePreparedStmt(ctx, stmt, []types.Datum{}) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + require.NoError(t, err) + ts, err := provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Equal(t, txnStartTS, ts) + + // second ts should reuse the txn startTS + rs, err = tk.Session().ExecutePreparedStmt(ctx, stmt, []types.Datum{}) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + require.NoError(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Equal(t, txnStartTS, ts) + + tk2.MustExec("update t set v = v + 10 where id = 1") + compareTS := getOracleTS(t, se) + rs, err = tk.Session().ExecutePreparedStmt(ctx, stmt, []types.Datum{}) + require.NoError(t, err) + _, err = session.ResultSetToStringSlice(ctx, tk.Session(), rs) + require.Error(t, err) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Greater(t, compareTS, ts) + // retry + tk.Session().GetSessionVars().RetryInfo.Retrying = true + rs, err = tk.Session().ExecutePreparedStmt(ctx, stmt, []types.Datum{}) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 11")) + ts, err = provider.GetStmtForUpdateTS() + require.NoError(t, err) + require.Greater(t, ts, compareTS) +} + func TestPessimisticRCTxnContextProviderLockError(t *testing.T) { store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() @@ -155,7 +211,7 @@ func TestPessimisticRCTxnContextProviderLockError(t *testing.T) { &tikverr.ErrDeadlock{Deadlock: &kvrpcpb.Deadlock{}, IsRetryable: true}, } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) @@ -167,7 +223,7 @@ func TestPessimisticRCTxnContextProviderLockError(t *testing.T) { errors.New("err"), } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Same(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) @@ -189,7 +245,7 @@ func TestPessimisticRCTxnContextProviderTS(t *testing.T) { // first read require.NoError(t, executor.ResetContextOfStmt(se, stmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) readTS, err := provider.GetStmtReadTS() require.NoError(t, err) forUpdateTS, err := provider.GetStmtForUpdateTS() @@ -202,7 +258,7 @@ func TestPessimisticRCTxnContextProviderTS(t *testing.T) { compareTS = getOracleTS(t, se) require.Greater(t, compareTS, readTS) require.NoError(t, executor.ResetContextOfStmt(se, stmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), stmt)) readTS, err = provider.GetStmtReadTS() require.NoError(t, err) forUpdateTS, err = provider.GetStmtForUpdateTS() @@ -267,7 +323,7 @@ func TestRCProviderInitialize(t *testing.T) { assertAfterActive := activeRCTxnAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider := assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -280,7 +336,7 @@ func TestRCProviderInitialize(t *testing.T) { assertAfterActive = activeRCTxnAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -346,12 +402,12 @@ func TestTidbSnapshotVarInRC(t *testing.T) { } // information schema and ts should equal to snapshot when tidb_snapshot is set - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() // information schema and ts will restore when set tidb_snapshot to empty tk.MustExec("set @@tidb_snapshot=''") - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseTxn(false) // txn will not be active after `GetStmtReadTS` or `GetStmtForUpdateTS` when `tidb_snapshot` is set @@ -372,7 +428,7 @@ func TestTidbSnapshotVarInRC(t *testing.T) { assertAfterUseSnapshot := activeSnapshotTxnAssert(se, se.GetSessionVars().SnapshotTS, "READ-COMMITTED") require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() assertAfterUseSnapshot.Check(t) }() diff --git a/sessiontxn/isolation/repeatable_read.go b/sessiontxn/isolation/repeatable_read.go index 571d2754be9a3..4827446d27a6c 100644 --- a/sessiontxn/isolation/repeatable_read.go +++ b/sessiontxn/isolation/repeatable_read.go @@ -122,8 +122,8 @@ func (p *PessimisticRRTxnContextProvider) updateForUpdateTS() (err error) { } // OnStmtStart is the hook that should be called when a new statement started -func (p *PessimisticRRTxnContextProvider) OnStmtStart(ctx context.Context) error { - if err := p.baseTxnContextProvider.OnStmtStart(ctx); err != nil { +func (p *PessimisticRRTxnContextProvider) OnStmtStart(ctx context.Context, node ast.StmtNode) error { + if err := p.baseTxnContextProvider.OnStmtStart(ctx, node); err != nil { return err } diff --git a/sessiontxn/isolation/repeatable_read_test.go b/sessiontxn/isolation/repeatable_read_test.go index c60c1c3da560d..c1487a1bb0ae7 100644 --- a/sessiontxn/isolation/repeatable_read_test.go +++ b/sessiontxn/isolation/repeatable_read_test.go @@ -74,7 +74,7 @@ func TestPessimisticRRErrorHandle(t *testing.T) { nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) - err = provider.OnStmtStart(context.TODO()) + err = provider.OnStmtStart(context.TODO(), nil) // Unlike StmtRetry which uses forUpdateTS got in OnStmtErrorForNextAction, OnStmtStart will reset provider's forUpdateTS, // which leads GetStmtForUpdateTS to acquire the latest ts. compareTS2 = getOracleTS(t, se) @@ -111,7 +111,7 @@ func TestPessimisticRRErrorHandle(t *testing.T) { nextAction, err = provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.NoError(t, err) require.Equal(t, sessiontxn.StmtActionRetryReady, nextAction) - err = provider.OnStmtStart(context.TODO()) + err = provider.OnStmtStart(context.TODO(), nil) require.NoError(t, err) // Unlike StmtRetry which uses forUpdateTS got in OnStmtErrorForNextAction, OnStmtStart will reset provider's forUpdateTS, // which leads GetStmtForUpdateTS to acquire the latest ts. @@ -153,7 +153,7 @@ func TestRepeatableReadProviderTS(t *testing.T) { compareTS := getOracleTS(t, se) // The read ts should be less than the compareTS require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) CurrentTS, err = provider.GetStmtReadTS() require.NoError(t, err) require.Greater(t, compareTS, CurrentTS) @@ -161,7 +161,7 @@ func TestRepeatableReadProviderTS(t *testing.T) { // The read ts should also be less than the compareTS in a new statement (after calling OnStmtStart) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) CurrentTS, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, CurrentTS, prevTS) @@ -175,14 +175,14 @@ func TestRepeatableReadProviderTS(t *testing.T) { // The for update read ts should be larger than the compareTS require.NoError(t, executor.ResetContextOfStmt(se, forUpdateStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) forUpdateTS, err := provider.GetStmtForUpdateTS() require.NoError(t, err) require.Greater(t, forUpdateTS, compareTS) // But the read ts is still less than the compareTS require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) CurrentTS, err = provider.GetStmtReadTS() require.NoError(t, err) require.Equal(t, CurrentTS, prevTS) @@ -228,7 +228,7 @@ func TestRepeatableReadProviderInitialize(t *testing.T) { assertAfterActive := activePessimisticRRAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider := assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -241,7 +241,7 @@ func TestRepeatableReadProviderInitialize(t *testing.T) { assertAfterActive = activePessimisticRRAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -303,12 +303,12 @@ func TestTidbSnapshotVarInPessimisticRepeatableRead(t *testing.T) { } // information schema and ts should equal to snapshot when tidb_snapshot is set - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() // information schema and ts will restore when set tidb_snapshot to empty tk.MustExec("set @@tidb_snapshot=''") - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseTxn() // txn will not be active after `GetStmtReadTS` or `GetStmtForUpdateTS` when `tidb_snapshot` is set @@ -329,7 +329,7 @@ func TestTidbSnapshotVarInPessimisticRepeatableRead(t *testing.T) { assertAfterUseSnapshot := activeSnapshotTxnAssert(se, se.GetSessionVars().SnapshotTS, "REPEATABLE-READ") require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() assertAfterUseSnapshot.Check(t) }() @@ -390,11 +390,11 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { for _, c := range cases { compareTS = getOracleTS(t, se) - require.NoError(t, txnManager.OnStmtStart(context.TODO())) + require.NoError(t, txnManager.OnStmtStart(context.TODO(), nil)) stmt, err = parser.New().ParseOneStmt(c.sql, "", "") require.NoError(t, err) - err = provider.OnStmtStart(context.TODO()) + err = provider.OnStmtStart(context.TODO(), nil) require.NoError(t, err) compiler = executor.Compiler{Ctx: se} @@ -432,9 +432,9 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { // Test use startTS after optimize when autocommit=0 activeAssert := activePessimisticRRAssert(t, tk.Session(), true) provider = initializeRepeatableReadProvider(t, tk, false) - require.NoError(t, txnManager.OnStmtStart(context.TODO())) stmt, err = parser.New().ParseOneStmt("update t set v = v + 10 where id = 1", "", "") require.NoError(t, err) + require.NoError(t, txnManager.OnStmtStart(context.TODO(), stmt)) execStmt, err = compiler.Compile(context.TODO(), stmt) require.NoError(t, err) err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) @@ -448,9 +448,9 @@ func TestOptimizeWithPlanInPessimisticRR(t *testing.T) { compareTS = getOracleTS(t, se) activeAssert = activePessimisticRRAssert(t, tk.Session(), true) provider = initializeRepeatableReadProvider(t, tk, false) - require.NoError(t, txnManager.OnStmtStart(context.TODO())) stmt, err = parser.New().ParseOneStmt("select * from t", "", "") require.NoError(t, err) + require.NoError(t, txnManager.OnStmtStart(context.TODO(), stmt)) execStmt, err = compiler.Compile(context.TODO(), stmt) require.NoError(t, err) err = txnManager.AdviseOptimizeWithPlan(execStmt.Plan) diff --git a/sessiontxn/isolation/serializable_test.go b/sessiontxn/isolation/serializable_test.go index a28e455195cbf..f192adf909369 100644 --- a/sessiontxn/isolation/serializable_test.go +++ b/sessiontxn/isolation/serializable_test.go @@ -54,7 +54,7 @@ func TestPessimisticSerializableTxnProviderTS(t *testing.T) { compareTS := getOracleTS(t, se) require.NoError(t, executor.ResetContextOfStmt(se, readOnlyStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) require.Greater(t, compareTS, ts) @@ -62,7 +62,7 @@ func TestPessimisticSerializableTxnProviderTS(t *testing.T) { // In Oracle-like serializable isolation, readTS equals to the for update ts require.NoError(t, executor.ResetContextOfStmt(se, forUpdateStmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err = provider.GetStmtForUpdateTS() require.NoError(t, err) require.Greater(t, compareTS, ts) @@ -87,7 +87,7 @@ func TestPessimisticSerializableTxnContextProviderLockError(t *testing.T) { &tikverr.ErrDeadlock{Deadlock: &kvrpcpb.Deadlock{}, IsRetryable: true}, } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Same(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) @@ -99,7 +99,7 @@ func TestPessimisticSerializableTxnContextProviderLockError(t *testing.T) { errors.New("err"), } { require.NoError(t, executor.ResetContextOfStmt(se, stmt)) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) nextAction, err := provider.OnStmtErrorForNextAction(sessiontxn.StmtErrAfterPessimisticLock, lockErr) require.Same(t, lockErr, err) require.Equal(t, sessiontxn.StmtActionError, nextAction) @@ -147,7 +147,7 @@ func TestSerializableInitialize(t *testing.T) { assertAfterActive := activeSerializableAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider := assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err := provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -160,7 +160,7 @@ func TestSerializableInitialize(t *testing.T) { assertAfterActive = activeSerializableAssert(t, se, true) require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) ts, err = provider.GetStmtReadTS() require.NoError(t, err) assertAfterActive.Check(t) @@ -223,12 +223,12 @@ func TestTidbSnapshotVarInSerialize(t *testing.T) { } // information schema and ts should equal to snapshot when tidb_snapshot is set - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() // information schema and ts will restore when set tidb_snapshot to empty tk.MustExec("set @@tidb_snapshot=''") - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseTxn() // txn will not be active after `GetStmtReadTS` or `GetStmtForUpdateTS` when `tidb_snapshot` is set @@ -250,7 +250,7 @@ func TestTidbSnapshotVarInSerialize(t *testing.T) { assertAfterUseSnapshot := activeSnapshotTxnAssert(se, se.GetSessionVars().SnapshotTS, "SERIALIZABLE") require.NoError(t, se.PrepareTxnCtx(context.TODO())) provider = assert.CheckAndGetProvider(t) - require.NoError(t, provider.OnStmtStart(context.TODO())) + require.NoError(t, provider.OnStmtStart(context.TODO(), nil)) checkUseSnapshot() assertAfterUseSnapshot.Check(t) }() diff --git a/sessiontxn/staleread/provider.go b/sessiontxn/staleread/provider.go index cc77cdd214b37..f76f500ed31c7 100644 --- a/sessiontxn/staleread/provider.go +++ b/sessiontxn/staleread/provider.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -86,7 +87,7 @@ func (p *StalenessTxnContextProvider) OnInitialize(ctx context.Context, tp sessi } // OnStmtStart is the hook that should be called when a new statement started -func (p *StalenessTxnContextProvider) OnStmtStart(_ context.Context) error { +func (p *StalenessTxnContextProvider) OnStmtStart(_ context.Context, _ ast.StmtNode) error { return nil } diff --git a/sessiontxn/txn_manager_test.go b/sessiontxn/txn_manager_test.go index e32f8bc2b3784..7fe8024b69de5 100644 --- a/sessiontxn/txn_manager_test.go +++ b/sessiontxn/txn_manager_test.go @@ -135,7 +135,7 @@ func TestEnterNewTxn(t *testing.T) { Type: sessiontxn.EnterNewTxnBeforeStmt, }) require.NoError(t, err) - require.NoError(t, mgr.OnStmtStart(context.TODO())) + require.NoError(t, mgr.OnStmtStart(context.TODO(), nil)) require.NoError(t, mgr.AdviseWarmup()) }, request: &sessiontxn.EnterNewTxnRequest{ diff --git a/tests/realtikvtest/sessiontest/temporary_table_test.go b/tests/realtikvtest/sessiontest/temporary_table_test.go index 6eb2ceddb5d3c..796f51cd68287 100644 --- a/tests/realtikvtest/sessiontest/temporary_table_test.go +++ b/tests/realtikvtest/sessiontest/temporary_table_test.go @@ -321,7 +321,7 @@ func TestTemporaryTableInterceptor(t *testing.T) { for _, initFunc := range initTxnFuncs { require.NoError(t, initFunc()) - require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO())) + require.NoError(t, sessiontxn.GetTxnManager(tk.Session()).OnStmtStart(context.TODO(), nil)) txn, err := tk.Session().Txn(true) require.NoError(t, err) diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index 5a0d39361f1ab..2db32927705f9 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -173,6 +173,9 @@ type Statement interface { // RebuildPlan rebuilds the plan of the statement. RebuildPlan(ctx context.Context) (schemaVersion int64, err error) + + // GetStmtNode returns the stmtNode inside Statement + GetStmtNode() ast.StmtNode } // RecordSet is an abstract result set interface to help get data from Plan. From 680b78cc9805b13ce05a4e6f6a00b0133edc744f Mon Sep 17 00:00:00 2001 From: Obliviate <756541536@qq.com> Date: Wed, 29 Jun 2022 16:38:39 +0800 Subject: [PATCH 25/28] lightning: revert new policy of allocating rowid and refine it later (#35817) close pingcap/tidb#35816 --- Makefile | 8 -- br/pkg/lightning/backend/kv/sql2kv.go | 48 -------- br/pkg/lightning/backend/local/engine.go | 51 +------- br/pkg/lightning/backend/local/engine_test.go | 111 ------------------ br/pkg/lightning/mydump/region.go | 63 ---------- br/pkg/lightning/mydump/region_test.go | 89 -------------- .../lightning/restore/chunk_restore_test.go | 2 +- br/pkg/lightning/restore/meta_manager.go | 56 --------- br/pkg/lightning/restore/meta_manager_test.go | 32 ----- br/pkg/lightning/restore/restore.go | 84 +++---------- br/pkg/lightning/restore/table_restore.go | 35 +----- .../lightning/restore/table_restore_test.go | 59 +++++----- br/tests/lightning_auto_random_default/run.sh | 8 +- br/tests/lightning_realloc_id/config.toml | 3 - br/tests/lightning_realloc_id/config1.toml | 2 - br/tests/lightning_realloc_id/config2.toml | 3 - .../data/db-schema-create.sql | 1 - .../data/db.test-schema.sql | 4 - .../data/db.test.000000000.csv | 11 -- .../data/db.test.000000001.sql | 11 -- .../data1/db-schema-create.sql | 1 - .../data1/db.test-schema.sql | 4 - .../data1/db.test.000000000.csv | 11 -- .../data1/db.test.000000001.sql | 11 -- .../data2/db.test.000000000.csv | 11 -- br/tests/lightning_realloc_id/run.sh | 93 --------------- 26 files changed, 50 insertions(+), 762 deletions(-) delete mode 100644 br/tests/lightning_realloc_id/config.toml delete mode 100644 br/tests/lightning_realloc_id/config1.toml delete mode 100644 br/tests/lightning_realloc_id/config2.toml delete mode 100644 br/tests/lightning_realloc_id/data/db-schema-create.sql delete mode 100644 br/tests/lightning_realloc_id/data/db.test-schema.sql delete mode 100644 br/tests/lightning_realloc_id/data/db.test.000000000.csv delete mode 100644 br/tests/lightning_realloc_id/data/db.test.000000001.sql delete mode 100644 br/tests/lightning_realloc_id/data1/db-schema-create.sql delete mode 100644 br/tests/lightning_realloc_id/data1/db.test-schema.sql delete mode 100644 br/tests/lightning_realloc_id/data1/db.test.000000000.csv delete mode 100644 br/tests/lightning_realloc_id/data1/db.test.000000001.sql delete mode 100644 br/tests/lightning_realloc_id/data2/db.test.000000000.csv delete mode 100644 br/tests/lightning_realloc_id/run.sh diff --git a/Makefile b/Makefile index 2314bcff43e63..cd5c7c5555027 100644 --- a/Makefile +++ b/Makefile @@ -327,14 +327,6 @@ build_for_br_integration_test: ) || (make failpoint-disable && exit 1) @make failpoint-disable -build_for_lightning_test: - @make failpoint-enable - $(GOTEST) -c -cover -covermode=count \ - -coverpkg=github.com/pingcap/tidb/br/... \ - -o $(LIGHTNING_BIN).test \ - github.com/pingcap/tidb/br/cmd/tidb-lightning - @make failpoint-disable - br_unit_test: export ARGS=$$($(BR_PACKAGES)) br_unit_test: @make failpoint-enable diff --git a/br/pkg/lightning/backend/kv/sql2kv.go b/br/pkg/lightning/backend/kv/sql2kv.go index bd13f27e38954..8a10ce607d9f7 100644 --- a/br/pkg/lightning/backend/kv/sql2kv.go +++ b/br/pkg/lightning/backend/kv/sql2kv.go @@ -450,49 +450,6 @@ func isPKCol(colInfo *model.ColumnInfo) bool { return mysql.HasPriKeyFlag(colInfo.GetFlag()) } -func isRowIDOverflow(meta *model.ColumnInfo, rowID int64) bool { - isUnsigned := mysql.HasUnsignedFlag(meta.GetFlag()) - switch meta.GetType() { - // MEDIUM INT - case mysql.TypeInt24: - if !isUnsigned { - return rowID > mysql.MaxInt24 - } - return rowID > mysql.MaxUint24 - // INT - case mysql.TypeLong: - if !isUnsigned { - return rowID > math.MaxInt32 - } - return rowID > math.MaxUint32 - // SMALLINT - case mysql.TypeShort: - if !isUnsigned { - return rowID > math.MaxInt16 - } - return rowID > math.MaxUint16 - // TINYINT - case mysql.TypeTiny: - if !isUnsigned { - return rowID > math.MaxInt8 - } - return rowID > math.MaxUint8 - // FLOAT - case mysql.TypeFloat: - if !isUnsigned { - return float32(rowID) > math.MaxFloat32 - } - return float64(rowID) > math.MaxFloat32*2 - // DOUBLE - case mysql.TypeDouble: - if !isUnsigned { - return float64(rowID) > math.MaxFloat64 - } - // impossible for rowID exceeding MaxFloat64 - } - return false -} - func (kvcodec *tableKVEncoder) getActualDatum(rowID int64, colIndex int, inputDatum *types.Datum) (types.Datum, error) { var ( value types.Datum @@ -520,11 +477,6 @@ func (kvcodec *tableKVEncoder) getActualDatum(rowID int64, colIndex int, inputDa // handle special values switch { case isAutoIncCol(col.ToInfo()): - // rowID is going to auto-filled the omitted column, - // which should be checked before restore - if isRowIDOverflow(col.ToInfo(), rowID) { - return value, errors.Errorf("PK %d is out of range", rowID) - } // we still need a conversion, e.g. to catch overflow with a TINYINT column. value, err = table.CastValue(kvcodec.se, types.NewIntDatum(rowID), col.ToInfo(), false, false) case isTableAutoRandom(tblMeta) && isPKCol(col.ToInfo()): diff --git a/br/pkg/lightning/backend/local/engine.go b/br/pkg/lightning/backend/local/engine.go index 04036e57b16ac..ad0b37cd01963 100644 --- a/br/pkg/lightning/backend/local/engine.go +++ b/br/pkg/lightning/backend/local/engine.go @@ -33,7 +33,6 @@ import ( "github.com/google/btree" "github.com/google/uuid" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" @@ -1003,21 +1002,6 @@ type Writer struct { batchSize int64 lastMetaSeq int32 - prevRowID int64 // only used for appendRowsSorted -} - -func (w *Writer) flushAndNewWriter() error { - var err error - err = w.flush(context.Background()) - if err != nil { - return errors.Trace(err) - } - newWriter, err := w.createSSTWriter() - if err != nil { - return errors.Trace(err) - } - w.writer = newWriter - return nil } func (w *Writer) appendRowsSorted(kvs []common.KvPair) error { @@ -1028,17 +1012,6 @@ func (w *Writer) appendRowsSorted(kvs []common.KvPair) error { } w.writer = writer } - if len(kvs) == 0 { - return nil - } - if w.prevRowID != 0 && kvs[0].RowID > w.prevRowID+1 { - // rowID leap. probably re-alloc id - // should write to different sst - err := w.flushAndNewWriter() - if err != nil { - return err - } - } keyAdapter := w.engine.keyAdapter totalKeySize := 0 @@ -1063,26 +1036,7 @@ func (w *Writer) appendRowsSorted(kvs []common.KvPair) error { } kvs = newKvs } - startIdx := 0 - w.prevRowID = kvs[len(kvs)-1].RowID - for i := 1; i < len(kvs); i++ { - if kvs[i].RowID > kvs[i-1].RowID+1 { - // leap id - err := w.writer.writeKVs(kvs[startIdx:i]) - if err != nil { - return err - } - err = w.flushAndNewWriter() - if err != nil { - return err - } - startIdx = i - } - } - if startIdx < len(kvs) { - return w.writer.writeKVs(kvs[startIdx:]) - } - return nil + return w.writer.writeKVs(kvs) } func (w *Writer) appendRowsUnsorted(ctx context.Context, kvs []common.KvPair) error { @@ -1149,9 +1103,6 @@ func (w *Writer) AppendRows(ctx context.Context, tableName string, columnNames [ } func (w *Writer) flush(ctx context.Context) error { - failpoint.Inject("MockFlushWriter", func() { - failpoint.Return(nil) - }) w.Lock() defer w.Unlock() if w.batchCount == 0 { diff --git a/br/pkg/lightning/backend/local/engine_test.go b/br/pkg/lightning/backend/local/engine_test.go index 13c890c028297..a0d8592d5398f 100644 --- a/br/pkg/lightning/backend/local/engine_test.go +++ b/br/pkg/lightning/backend/local/engine_test.go @@ -29,9 +29,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/stretchr/testify/require" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/backend" - "github.com/pingcap/tidb/br/pkg/lightning/common" ) func TestIngestSSTWithClosedEngine(t *testing.T) { @@ -87,112 +85,3 @@ func TestIngestSSTWithClosedEngine(t *testing.T) { }, }), errorEngineClosed) } - -func TestAutoSplitSST(t *testing.T) { - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/MockFlushWriter", "return(true)")) - defer func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/MockFlushWriter")) - }() - var err error - dir := os.TempDir() - w := &Writer{ - engine: &Engine{ - sstDir: dir, - keyAdapter: noopKeyAdapter{}, - logger: log.L(), - }, - isKVSorted: true, - isWriteBatchSorted: true, - } - w.engine.closed.Store(false) - w.writer, err = w.createSSTWriter() - require.Nil(t, err) - kvs := []common.KvPair{ - { - Key: []byte("1"), - Val: []byte("val1"), - RowID: 1, - }, - { - Key: []byte("2"), - Val: []byte("val1"), - RowID: 2, - }, - } - prevWriter := w.writer - err = w.appendRowsSorted(kvs) - require.Nil(t, err) - require.True(t, prevWriter == w.writer) - kvs = []common.KvPair{ - { - Key: []byte("10"), - Val: []byte("val10"), - RowID: 10, - }, - { - Key: []byte("11"), - Val: []byte("val11"), - RowID: 11, - }, - } - err = w.appendRowsSorted(kvs) - require.Nil(t, err) - require.False(t, prevWriter == w.writer) // id leap, should flush and create - prevWriter = w.writer - kvs = []common.KvPair{ - { - Key: []byte("12"), - Val: []byte("val12"), - RowID: 10, - }, - { - Key: []byte("13"), - Val: []byte("val13"), - RowID: 11, - }, - { - Key: []byte("15"), - Val: []byte("val15"), - RowID: 15, - }, - } - err = w.appendRowsSorted(kvs) - require.Nil(t, err) - require.False(t, prevWriter == w.writer) // id leap, should flush and create - prevWriter = w.writer - kvs = []common.KvPair{ - { - Key: []byte("16"), - Val: []byte("val16"), - RowID: 16, - }, - { - Key: []byte("17"), - Val: []byte("val17"), - RowID: 17, - }, - { - Key: []byte("19"), - Val: []byte("val19"), - RowID: 19, - }, - { - Key: []byte("20"), - Val: []byte("val20"), - RowID: 20, - }, - { - Key: []byte("22"), - Val: []byte("val22"), - RowID: 22, - }, - { - Key: []byte("23"), - Val: []byte("val23"), - RowID: 22, - }, - } - err = w.appendRowsSorted(kvs) - require.Nil(t, err) - require.False(t, prevWriter == w.writer) // id leap, should flush and create -} diff --git a/br/pkg/lightning/mydump/region.go b/br/pkg/lightning/mydump/region.go index 04cc75e5567ae..b4f2537fb2507 100644 --- a/br/pkg/lightning/mydump/region.go +++ b/br/pkg/lightning/mydump/region.go @@ -22,7 +22,6 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/worker" @@ -272,12 +271,6 @@ func makeSourceFileRegion( if !isCsvFile { divisor += 2 } - sizePerRow, err := GetSampledAvgRowSize(&fi, cfg, ioWorkers, store) - if err == nil && sizePerRow != 0 { - log.FromContext(ctx).Warn("fail to sample file", zap.String("path", fi.FileMeta.Path), zap.Error(err)) - divisor = sizePerRow - } - log.FromContext(ctx).Debug("avg row size", zap.String("path", fi.FileMeta.Path), zap.Int64("size per row", sizePerRow)) // If a csv file is overlarge, we need to split it into multiple regions. // Note: We can only split a csv file whose format is strict. // We increase the check threshold by 1/10 of the `max-region-size` because the source file size dumped by tools @@ -299,10 +292,6 @@ func makeSourceFileRegion( RowIDMax: fi.FileMeta.FileSize / divisor, }, } - failpoint.Inject("MockInaccurateRowID", func() { - // only allocates 5 rows but contains 10 rows - tableRegion.Chunk.RowIDMax = 5 - }) if tableRegion.Size() > tableRegionSizeWarningThreshold { log.FromContext(ctx).Warn( @@ -313,55 +302,6 @@ func makeSourceFileRegion( return []*TableRegion{tableRegion}, []float64{float64(fi.FileMeta.FileSize)}, nil } -func GetSampledAvgRowSize( - fileInfo *FileInfo, - cfg *config.Config, - ioWorkers *worker.Pool, - store storage.ExternalStorage, -) (int64, error) { - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - defer cancel() - reader, err := store.Open(ctx, fileInfo.FileMeta.Path) - if err != nil { - return 0, err - } - var parser Parser - switch fileInfo.FileMeta.Type { - case SourceTypeCSV: - hasHeader := cfg.Mydumper.CSV.Header - charsetConvertor, err := NewCharsetConvertor(cfg.Mydumper.DataCharacterSet, cfg.Mydumper.DataInvalidCharReplace) - if err != nil { - return 0, err - } - parser, err = NewCSVParser(ctx, &cfg.Mydumper.CSV, reader, int64(cfg.Mydumper.ReadBlockSize), ioWorkers, hasHeader, charsetConvertor) - if err != nil { - return 0, err - } - case SourceTypeSQL: - parser = NewChunkParser(ctx, cfg.TiDB.SQLMode, reader, int64(cfg.Mydumper.ReadBlockSize), ioWorkers) - default: - return 0, errors.Errorf("source file %s is none of csv, sql, or parquet file", fileInfo.FileMeta.Path) - } - totalBytes := 0 - totalRows := 0 - defaultSampleRows := 10 // todo: may be configurable - for i := 0; i < defaultSampleRows; i++ { - err = parser.ReadRow() - if err != nil && errors.Cause(err) == io.EOF { - break - } else if err != nil { - return 0, err - } - totalBytes += parser.LastRow().Length - totalRows++ - } - if totalRows > 0 { - return int64(totalBytes) / int64(totalRows), nil - } else { - return 0, nil - } -} - // because parquet files can't seek efficiently, there is no benefit in split. // parquet file are column orient, so the offset is read line number func makeParquetFileRegion( @@ -441,9 +381,6 @@ func SplitLargeFile( } for { curRowsCnt := (endOffset - startOffset) / divisor - if curRowsCnt == 0 && endOffset != startOffset { - curRowsCnt = 1 - } rowIDMax := prevRowIdxMax + curRowsCnt if endOffset != dataFile.FileMeta.FileSize { r, err := store.Open(ctx, dataFile.FileMeta.Path) diff --git a/br/pkg/lightning/mydump/region_test.go b/br/pkg/lightning/mydump/region_test.go index 37ba4e4028e39..a1dbb9f290a69 100644 --- a/br/pkg/lightning/mydump/region_test.go +++ b/br/pkg/lightning/mydump/region_test.go @@ -40,8 +40,6 @@ import ( */ func TestTableRegion(t *testing.T) { cfg := newConfigWithSourceDir("./examples") - // specify ReadBlockSize because we need to sample files - cfg.Mydumper.ReadBlockSize = config.ReadBlockSize loader, _ := NewMyDumpLoader(context.Background(), cfg) dbMeta := loader.GetDatabases()[0] @@ -384,90 +382,3 @@ func TestSplitLargeFileOnlyOneChunk(t *testing.T) { require.Equal(t, columns, regions[i].Chunk.Columns) } } - -func TestSampleAndGetAvgRowSize(t *testing.T) { - // It's more difficult to estimate sizes of SQL files than csv files, - // because when reading the first row of them, parser may read other info (e.g. table name) - // so that make it hard to get good estimate, especially when files have few rows. - sqlFiles := []string{ - // 1. long table name, values: - // 1.1 short and even len - "INSERT INTO `test_db_mock_long.test_table_very_long_name` VALUES (1),(2);", - // 1.2 short and not even - "INSERT INTO `test_db_mock_long.test_table_very_long_name` VALUES (123452123,1234123125),(2,1);", - "INSERT INTO `test_db_mock_long.test_table_very_long_name` VALUES (2,1),(123452123,1234123125);", - // 1.3 long and even - "INSERT INTO `test_db_mock_long.test_table_very_long_name` VALUES (123452123,1234123125),(1234123125,12341231251);", - // 1.4 long but not even - "INSERT INTO `test_db_mock_long.test_table_very_long_name` VALUES ('abcdefghidgjla','lkjadsfasfdkjl'),('1111111','1');", - // 2. short table name, values: - // 2.1 short and even len - "INSERT INTO `a` VALUES (1),(2);", - // 2.2 short and not even - "INSERT INTO `a` VALUES (123452123,1234123125),(2,1);", - "INSERT INTO `a` VALUES (2,1),(123452123,1234123125);", - // 2.3 long and even - "INSERT INTO `a` VALUES (123452123,1234123125),(1234123125,12341231251);", - // 2.4 long but not even - "INSERT INTO `a` VALUES ('abcdefghidgjla','lkjadsfasfdkjl'),('1111111','1');", - } - - csvFiles := []string{ - // even and short - "a,b,c\r\n1,2,3\r\n4,5,6\r\n", - // not even but short - "a,b,c\r\n1112,1234,1923\r\n1,2,3", - // even and long - "a,b,c\r\n14712312,123122,1231233\r\n4456364,34525,423426\r\n", - // not even but long - "a,b,c\r\nsadlk;fja;lskdfj;alksdfj,sdlk;fjaksld;fja;l,qpoiwuepqou\r\n0,0,0\r\n", - } - testFunc := func(files []string, fileType SourceType) { - for _, file := range files { - dir := t.TempDir() - - var fileName string - if fileType == SourceTypeCSV { - fileName = "test.csv" - } else { - fileName = "test.sql" - } - filePath := filepath.Join(dir, fileName) - - content := []byte(file) - err := os.WriteFile(filePath, content, 0o644) - require.Nil(t, err) - dataFileInfo, err := os.Stat(filePath) - require.Nil(t, err) - fileSize := dataFileInfo.Size() - - cfg := newConfigWithSourceDir(dir) - loader, _ := NewMyDumpLoader(context.Background(), cfg) - ioWorkers := worker.NewPool(context.Background(), 1, "io") - - // specify ReadBlockSize because we need to sample files - cfg.Mydumper.ReadBlockSize = config.ReadBlockSize - fileInfo := FileInfo{ - FileMeta: SourceFileMeta{ - Path: fileName, - Type: fileType, - FileSize: fileSize, - }, - } - cfg.Mydumper.CSV = config.CSVConfig{ - Separator: ",", - Delimiter: `"`, - Header: true, - NotNull: false, - Null: `\N`, - BackslashEscape: true, - TrimLastSep: false, - } - size, err := GetSampledAvgRowSize(&fileInfo, cfg, ioWorkers, loader.GetStore()) - require.Nil(t, err) - require.GreaterOrEqual(t, fileSize/size, int64(2)) - } - } - testFunc(sqlFiles, SourceTypeSQL) - testFunc(csvFiles, SourceTypeCSV) -} diff --git a/br/pkg/lightning/restore/chunk_restore_test.go b/br/pkg/lightning/restore/chunk_restore_test.go index 59d083d85561c..2a9a42434c77b 100644 --- a/br/pkg/lightning/restore/chunk_restore_test.go +++ b/br/pkg/lightning/restore/chunk_restore_test.go @@ -73,7 +73,7 @@ func (s *chunkRestoreSuite) SetupTest() { } var err error - s.cr, err = newChunkRestore(context.Background(), 1, s.cfg, &chunk, w, s.store, nil, nil) + s.cr, err = newChunkRestore(context.Background(), 1, s.cfg, &chunk, w, s.store, nil) require.NoError(s.T(), err) } diff --git a/br/pkg/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index b94bde8208be6..0bac9bd436613 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -78,11 +78,6 @@ func (b *dbMetaMgrBuilder) TableMetaMgr(tr *TableRestore) tableMetaMgr { type tableMetaMgr interface { InitTableMeta(ctx context.Context) error AllocTableRowIDs(ctx context.Context, rawRowIDMax int64) (*verify.KVChecksum, int64, error) - // ReallocTableRowIDs reallocates the row IDs of a table. - // It returns new rowIDBase and maxRowID or any error it encounters. - // Note that noopTableMetaMgr has a noop implementation of this function. - // If maxRowID is 0, caller should maintain rowIDBase and maxRowID itself. - ReallocTableRowIDs(ctx context.Context, newRowIDCount int64) (int64, int64, error) UpdateTableStatus(ctx context.Context, status metaStatus) error UpdateTableBaseChecksum(ctx context.Context, checksum *verify.KVChecksum) error CheckAndUpdateLocalChecksum(ctx context.Context, checksum *verify.KVChecksum, hasLocalDupes bool) ( @@ -165,51 +160,6 @@ func parseMetaStatus(s string) (metaStatus, error) { } } -func (m *dbTableMetaMgr) ReallocTableRowIDs(ctx context.Context, newRowIDCount int64) (int64, int64, error) { - conn, err := m.session.Conn(ctx) - if err != nil { - return 0, 0, errors.Trace(err) - } - defer conn.Close() - exec := &common.SQLWithRetry{ - DB: m.session, - Logger: m.tr.logger, - } - err = exec.Exec(ctx, "enable pessimistic transaction", "SET SESSION tidb_txn_mode = 'pessimistic';") - if err != nil { - return 0, 0, errors.Annotate(err, "enable pessimistic transaction failed") - } - var ( - maxRowIDMax int64 - newRowIDMax int64 - ) - err = exec.Transact(ctx, "realloc table rowID", func(ctx context.Context, tx *sql.Tx) error { - row := tx.QueryRowContext( - ctx, - fmt.Sprintf("SELECT MAX(row_id_max) from %s WHERE table_id = ? FOR UPDATE", m.tableName), - m.tr.tableInfo.ID, - ) - if row.Err() != nil { - return errors.Trace(err) - } - if err := row.Scan(&maxRowIDMax); err != nil { - return errors.Trace(err) - } - newRowIDMax = maxRowIDMax + newRowIDCount - // nolint:gosec - query := fmt.Sprintf("UPDATE %s SET row_id_max = ? WHERE table_id = ? AND task_id = ?", m.tableName) - if _, err := tx.ExecContext(ctx, query, newRowIDMax, m.tr.tableInfo.ID, m.taskID); err != nil { - return err - } - return nil - }) - if err != nil { - return 0, 0, errors.Trace(err) - } - // newRowIDBase = maxRowIDMax + 1 - return maxRowIDMax + 1, newRowIDMax, nil -} - func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64) (*verify.KVChecksum, int64, error) { conn, err := m.session.Conn(ctx) if err != nil { @@ -1097,12 +1047,6 @@ func (m noopTableMetaMgr) InitTableMeta(ctx context.Context) error { return nil } -func (m noopTableMetaMgr) ReallocTableRowIDs(ctx context.Context, _ int64) (int64, int64, error) { - // we don't need to reconcile rowIDs across all the instances - // barring using parallel import - return 0, 0, nil -} - func (m noopTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64) (*verify.KVChecksum, int64, error) { return nil, 0, nil } diff --git a/br/pkg/lightning/restore/meta_manager_test.go b/br/pkg/lightning/restore/meta_manager_test.go index 23102b56f07a6..8480bf077d6de 100644 --- a/br/pkg/lightning/restore/meta_manager_test.go +++ b/br/pkg/lightning/restore/meta_manager_test.go @@ -384,35 +384,3 @@ func TestSingleTaskMetaMgr(t *testing.T) { }) require.NoError(t, err) } - -func TestReallocTableRowIDs(t *testing.T) { - s, clean := newMetaMgrSuite(t) - defer clean() - - ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) - - rows := [][]driver.Value{ - {int64(1), int64(998), int64(1008), uint64(0), uint64(0), uint64(0), metaStatusRowIDAllocated.String()}, - } - checksum := verification.MakeKVChecksum(2, 1, 3) - s.prepareMock(rows, nil, nil, &checksum, nil) - - ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - require.NoError(t, err) - require.Equal(t, int64(998), rowIDBase) - require.Equal(t, &checksum, ck) - require.Equal(t, 1, s.checksumMgr.callCnt) - s.mockDB.ExpectExec("SET SESSION tidb_txn_mode = 'pessimistic';"). - WillReturnResult(sqlmock.NewResult(int64(0), int64(0))) - - s.mockDB.ExpectBegin() - s.mockDB.ExpectQuery("\\QSELECT MAX(row_id_max) from `test`.`table_meta` WHERE table_id = ? FOR UPDATE\\E").WithArgs(int64(1)). - WillReturnRows(sqlmock.NewRows([]string{"row_id_max"}).AddRow(1008)) - s.mockDB.ExpectExec("\\QUPDATE `test`.`table_meta` SET row_id_max = ? WHERE table_id = ? AND task_id = ?\\E").WithArgs(int64(1018), int64(1), int64(1)). - WillReturnResult(sqlmock.NewResult(int64(0), int64(1))) - s.mockDB.ExpectCommit() - newBase, newMax, err := s.mgr.ReallocTableRowIDs(context.Background(), 10) - require.Nil(t, err) - require.Equal(t, int64(1009), newBase) - require.Equal(t, int64(1018), newMax) -} diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index faf774efa5c7a..0a43936ae5661 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -1610,9 +1610,6 @@ func (tr *TableRestore) restoreTable( rowIDMax = engine.Chunks[len(engine.Chunks)-1].Chunk.RowIDMax } } - if rowIDMax > tr.curMaxRowID { - tr.curMaxRowID = rowIDMax - } db, _ := rc.tidbGlue.GetDB() versionStr, err := version.FetchVersion(ctx, db) if err != nil { @@ -2055,16 +2052,9 @@ func (rc *Controller) DataCheck(ctx context.Context) error { } type chunkRestore struct { - parser mydump.Parser - index int - chunk *checkpoints.ChunkCheckpoint - originalRowIDMax int64 - curRowIDBase int64 - curRowIDMax int64 - tableRestore *TableRestore - - rowCount int - curAccmRowSize uint64 // has a maximum of 18446744.07370955 TB + parser mydump.Parser + index int + chunk *checkpoints.ChunkCheckpoint } func newChunkRestore( @@ -2075,7 +2065,6 @@ func newChunkRestore( ioWorkers *worker.Pool, store storage.ExternalStorage, tableInfo *checkpoints.TidbTableInfo, - tableRestore *TableRestore, ) (*chunkRestore, error) { blockBufSize := int64(cfg.Mydumper.ReadBlockSize) @@ -2122,11 +2111,9 @@ func newChunkRestore( } return &chunkRestore{ - parser: parser, - index: index, - chunk: chunk, - originalRowIDMax: chunk.Chunk.RowIDMax, - tableRestore: tableRestore, + parser: parser, + index: index, + chunk: chunk, }, nil } @@ -2179,52 +2166,13 @@ type deliverResult struct { err error } -func (cr *chunkRestore) adjustRowID(rowID int64, rc *Controller) (int64, error) { - if rowID <= cr.originalRowIDMax { - // no need to ajust - return rowID, nil - } - // need to adjust rowID - // rowID should be within [curRowIDBase, curRowIDMax] - if cr.curRowIDBase == 0 || cr.curRowIDBase > cr.curRowIDMax { - logger := cr.tableRestore.logger.With( - zap.String("tableName", cr.tableRestore.tableName), - zap.Int("fileIndex", cr.index), - zap.Stringer("path", &cr.chunk.Key), - zap.String("task", "re-allocate rowID"), - ) - logger.Info("start re-allocating") - // 1. curRowIDBase == 0 -> no previous re-allocation - // 2. curRowIDBase > curRowIDMax -> run out of allocated IDs - pos, _ := cr.parser.Pos() - leftFileSize := cr.chunk.Chunk.EndOffset - pos - avgRowSize := cr.curAccmRowSize / uint64(cr.rowCount) - newRowIDCount := leftFileSize/int64(avgRowSize) + 1 // plus the current row - newBase, newMax, err := cr.tableRestore.allocateRowIDs(newRowIDCount, rc) - if err != nil { - logger.Error("fail to re-allocate rowIDs", zap.Error(err)) - return 0, err - } - cr.curRowIDBase = newBase - cr.curRowIDMax = newMax - } - rowID = cr.curRowIDBase - cr.curRowIDBase++ - return rowID, nil -} - -func (cr *chunkRestore) updateRowStats(rowSize int) { - cr.curAccmRowSize += uint64(rowSize) - cr.rowCount++ -} - //nolint:nakedret // TODO: refactor func (cr *chunkRestore) deliverLoop( ctx context.Context, kvsCh <-chan []deliveredKVs, t *TableRestore, engineID int32, - dataWriter, indexWriter *backend.LocalEngineWriter, + dataEngine, indexEngine *backend.LocalEngineWriter, rc *Controller, ) (deliverTotalDur time.Duration, err error) { deliverLogger := t.logger.With( @@ -2248,6 +2196,7 @@ func (cr *chunkRestore) deliverLoop( startOffset := cr.chunk.Chunk.Offset currOffset := startOffset rowID := cr.chunk.Chunk.PrevRowIDMax + populate: for dataChecksum.SumSize()+indexChecksum.SumSize() < minDeliverBytes { select { @@ -2282,7 +2231,7 @@ func (cr *chunkRestore) deliverLoop( for !rc.diskQuotaLock.TryRLock() { // try to update chunk checkpoint, this can help save checkpoint after importing when disk-quota is triggered if !dataSynced { - dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataWriter, indexWriter) + dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataEngine, indexEngine) } time.Sleep(time.Millisecond) } @@ -2291,14 +2240,14 @@ func (cr *chunkRestore) deliverLoop( // Write KVs into the engine start := time.Now() - if err = dataWriter.WriteRows(ctx, columns, dataKVs); err != nil { + if err = dataEngine.WriteRows(ctx, columns, dataKVs); err != nil { if !common.IsContextCanceledError(err) { deliverLogger.Error("write to data engine failed", log.ShortError(err)) } return errors.Trace(err) } - if err = indexWriter.WriteRows(ctx, columns, indexKVs); err != nil { + if err = indexEngine.WriteRows(ctx, columns, indexKVs); err != nil { if !common.IsContextCanceledError(err) { deliverLogger.Error("write to index engine failed", log.ShortError(err)) } @@ -2340,7 +2289,7 @@ func (cr *chunkRestore) deliverLoop( if currOffset > lastOffset || dataChecksum.SumKVS() != 0 || indexChecksum.SumKVS() != 0 { // No need to save checkpoint if nothing was delivered. - dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataWriter, indexWriter) + dataSynced = cr.maybeSaveCheckpoint(rc, t, engineID, cr.chunk, dataEngine, indexEngine) } failpoint.Inject("SlowDownWriteRows", func() { deliverLogger.Warn("Slowed down write rows") @@ -2511,11 +2460,6 @@ func (cr *chunkRestore) encodeLoop( encodeDurStart := time.Now() lastRow := cr.parser.LastRow() // sql -> kv - if lastRow.RowID, err = cr.adjustRowID(lastRow.RowID, rc); err != nil { - return - } - cr.updateRowStats(lastRow.Length) - rowID = lastRow.RowID kvs, encodeErr := kvEncoder.Encode(logger, lastRow.Row, lastRow.RowID, cr.chunk.ColumnPermutation, cr.chunk.Key.Path, curOffset) encodeDur += time.Since(encodeDurStart) @@ -2578,7 +2522,7 @@ func (cr *chunkRestore) restore( ctx context.Context, t *TableRestore, engineID int32, - dataWriter, indexWriter *backend.LocalEngineWriter, + dataEngine, indexEngine *backend.LocalEngineWriter, rc *Controller, ) error { // Create the encoder. @@ -2599,7 +2543,7 @@ func (cr *chunkRestore) restore( go func() { defer close(deliverCompleteCh) - dur, err := cr.deliverLoop(ctx, kvsCh, t, engineID, dataWriter, indexWriter, rc) + dur, err := cr.deliverLoop(ctx, kvsCh, t, engineID, dataEngine, indexEngine, rc) select { case <-ctx.Done(): case deliverCompleteCh <- deliverResult{dur, err}: diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 491a59fa1c33b..7e44234f1d773 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -56,8 +56,6 @@ type TableRestore struct { logger log.Logger ignoreColumns map[string]struct{} - rowIDLock sync.Mutex - curMaxRowID int64 } func NewTableRestore( @@ -150,9 +148,6 @@ func (tr *TableRestore) RebaseChunkRowIDs(cp *checkpoints.TableCheckpoint, rowID for _, chunk := range engine.Chunks { chunk.Chunk.PrevRowIDMax += rowIDBase chunk.Chunk.RowIDMax += rowIDBase - if chunk.Chunk.RowIDMax > tr.curMaxRowID { - tr.curMaxRowID = chunk.Chunk.RowIDMax - } } } } @@ -510,7 +505,7 @@ func (tr *TableRestore) restoreEngine( // 2. sql -> kvs // 3. load kvs data (into kv deliver server) // 4. flush kvs data (into tikv node) - cr, err := newChunkRestore(ctx, chunkIndex, rc.cfg, chunk, rc.ioWorkers, rc.store, tr.tableInfo, tr) + cr, err := newChunkRestore(ctx, chunkIndex, rc.cfg, chunk, rc.ioWorkers, rc.store, tr.tableInfo) if err != nil { setError(err) break @@ -1053,31 +1048,3 @@ func estimateCompactionThreshold(cp *checkpoints.TableCheckpoint, factor int64) return threshold } - -func (tr *TableRestore) allocateRowIDs(newRowCount int64, rc *Controller) (int64, int64, error) { - tr.rowIDLock.Lock() - defer tr.rowIDLock.Unlock() - metaMgr := rc.metaMgrBuilder.TableMetaMgr(tr) - // try to re-allocate from downstream - // if we are using parallel import, rowID should be reconciled globally. - // Otherwise, this function will simply return 0. - newRowIDBase, newRowIDMax, err := metaMgr.ReallocTableRowIDs(context.Background(), newRowCount) - if err != nil { - return 0, 0, err - } - // TODO: refinement: currently, when we're not using SSTMode + incremental, - // metadata of the table restore is not maintained globally. - // So we have to deviate this two disparate situations here and make - // code complexer. - var rowIDBase int64 - if newRowIDMax != 0 { - // re-alloc from downstream - rowIDBase = newRowIDBase - tr.curMaxRowID = newRowIDMax - } else { - // single import mode: re-allocate rowID from memory - rowIDBase = tr.curMaxRowID + 1 - tr.curMaxRowID += newRowCount - } - return rowIDBase, tr.curMaxRowID, nil -} diff --git a/br/pkg/lightning/restore/table_restore_test.go b/br/pkg/lightning/restore/table_restore_test.go index 87aa389c7167b..5a8799ad43002 100644 --- a/br/pkg/lightning/restore/table_restore_test.go +++ b/br/pkg/lightning/restore/table_restore_test.go @@ -198,7 +198,6 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Engines: make(map[int32]*checkpoints.EngineCheckpoint), } - s.cfg.Mydumper.CSV.Header = false rc := &Controller{cfg: s.cfg, ioWorkers: worker.NewPool(context.Background(), 1, "io"), store: s.store} err := s.tr.populateChunks(context.Background(), rc, cp) require.NoError(s.T(), err) @@ -217,7 +216,7 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Offset: 0, EndOffset: 37, PrevRowIDMax: 0, - RowIDMax: 1, + RowIDMax: 7, // 37 bytes with 3 columns can store at most 7 rows. }, Timestamp: 1234567897, }, @@ -227,8 +226,8 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Chunk: mydump.Chunk{ Offset: 0, EndOffset: 37, - PrevRowIDMax: 1, - RowIDMax: 2, + PrevRowIDMax: 7, + RowIDMax: 14, }, Timestamp: 1234567897, }, @@ -238,8 +237,8 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Chunk: mydump.Chunk{ Offset: 0, EndOffset: 37, - PrevRowIDMax: 2, - RowIDMax: 3, + PrevRowIDMax: 14, + RowIDMax: 21, }, Timestamp: 1234567897, }, @@ -254,8 +253,8 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Chunk: mydump.Chunk{ Offset: 0, EndOffset: 37, - PrevRowIDMax: 3, - RowIDMax: 4, + PrevRowIDMax: 21, + RowIDMax: 28, }, Timestamp: 1234567897, }, @@ -265,8 +264,8 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Chunk: mydump.Chunk{ Offset: 0, EndOffset: 37, - PrevRowIDMax: 4, - RowIDMax: 5, + PrevRowIDMax: 28, + RowIDMax: 35, }, Timestamp: 1234567897, }, @@ -276,8 +275,8 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Chunk: mydump.Chunk{ Offset: 0, EndOffset: 37, - PrevRowIDMax: 5, - RowIDMax: 6, + PrevRowIDMax: 35, + RowIDMax: 42, }, Timestamp: 1234567897, }, @@ -292,8 +291,8 @@ func (s *tableRestoreSuite) TestPopulateChunks() { Chunk: mydump.Chunk{ Offset: 0, EndOffset: 14, - PrevRowIDMax: 6, - RowIDMax: 10, + PrevRowIDMax: 42, + RowIDMax: 46, }, Timestamp: 1234567897, }, @@ -473,7 +472,7 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Offset: 0, EndOffset: 14, PrevRowIDMax: 0, - RowIDMax: 4, // 14 bytes and 3 byte for each row + RowIDMax: 4, // 37 bytes with 3 columns can store at most 7 rows. }, Timestamp: 1234567897, }, @@ -484,7 +483,7 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Offset: 0, EndOffset: 10, PrevRowIDMax: 4, - RowIDMax: 9, // 10 bytes and 2 byte for each row + RowIDMax: 7, }, Timestamp: 1234567897, }, @@ -495,8 +494,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 6, EndOffset: 52, - PrevRowIDMax: 9, - RowIDMax: 13, + PrevRowIDMax: 7, + RowIDMax: 20, Columns: []string{"a", "b", "c"}, }, @@ -509,8 +508,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 52, EndOffset: 60, - PrevRowIDMax: 13, - RowIDMax: 14, + PrevRowIDMax: 20, + RowIDMax: 22, Columns: []string{"a", "b", "c"}, }, Timestamp: 1234567897, @@ -522,8 +521,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 6, EndOffset: 48, - PrevRowIDMax: 14, - RowIDMax: 17, + PrevRowIDMax: 22, + RowIDMax: 35, Columns: []string{"c", "a", "b"}, }, Timestamp: 1234567897, @@ -540,8 +539,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 48, EndOffset: 101, - PrevRowIDMax: 17, - RowIDMax: 20, + PrevRowIDMax: 35, + RowIDMax: 48, Columns: []string{"c", "a", "b"}, }, Timestamp: 1234567897, @@ -553,8 +552,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 101, EndOffset: 102, - PrevRowIDMax: 20, - RowIDMax: 21, + PrevRowIDMax: 48, + RowIDMax: 48, Columns: []string{"c", "a", "b"}, }, Timestamp: 1234567897, @@ -566,8 +565,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 4, EndOffset: 59, - PrevRowIDMax: 21, - RowIDMax: 23, + PrevRowIDMax: 48, + RowIDMax: 61, Columns: []string{"b", "c"}, }, Timestamp: 1234567897, @@ -584,8 +583,8 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { Chunk: mydump.Chunk{ Offset: 59, EndOffset: 60, - PrevRowIDMax: 23, - RowIDMax: 24, + PrevRowIDMax: 61, + RowIDMax: 61, Columns: []string{"b", "c"}, }, Timestamp: 1234567897, diff --git a/br/tests/lightning_auto_random_default/run.sh b/br/tests/lightning_auto_random_default/run.sh index c54ca6ac7ee0e..41b9798de4560 100644 --- a/br/tests/lightning_auto_random_default/run.sh +++ b/br/tests/lightning_auto_random_default/run.sh @@ -40,10 +40,10 @@ for backend in tidb local; do check_contains 'inc: 6' NEXT_AUTO_RAND_VAL=7 else - check_contains 'inc: 6' - check_contains 'inc: 7' - check_contains 'inc: 8' - NEXT_AUTO_RAND_VAL=9 + check_contains 'inc: 25' + check_contains 'inc: 26' + check_contains 'inc: 27' + NEXT_AUTO_RAND_VAL=28 fi # tidb backend randomly generate the auto-random bit for each statement, so with 2 statements, diff --git a/br/tests/lightning_realloc_id/config.toml b/br/tests/lightning_realloc_id/config.toml deleted file mode 100644 index f32593b43b798..0000000000000 --- a/br/tests/lightning_realloc_id/config.toml +++ /dev/null @@ -1,3 +0,0 @@ -[tikv-importer] -incremental-import = true -backend = 'local' diff --git a/br/tests/lightning_realloc_id/config1.toml b/br/tests/lightning_realloc_id/config1.toml deleted file mode 100644 index d2152b47c922a..0000000000000 --- a/br/tests/lightning_realloc_id/config1.toml +++ /dev/null @@ -1,2 +0,0 @@ -[tikv-importer] -backend = 'local' diff --git a/br/tests/lightning_realloc_id/config2.toml b/br/tests/lightning_realloc_id/config2.toml deleted file mode 100644 index f32593b43b798..0000000000000 --- a/br/tests/lightning_realloc_id/config2.toml +++ /dev/null @@ -1,3 +0,0 @@ -[tikv-importer] -incremental-import = true -backend = 'local' diff --git a/br/tests/lightning_realloc_id/data/db-schema-create.sql b/br/tests/lightning_realloc_id/data/db-schema-create.sql deleted file mode 100644 index c88b0e3150e76..0000000000000 --- a/br/tests/lightning_realloc_id/data/db-schema-create.sql +++ /dev/null @@ -1 +0,0 @@ -create database db; \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data/db.test-schema.sql b/br/tests/lightning_realloc_id/data/db.test-schema.sql deleted file mode 100644 index 0490cd81e1c2e..0000000000000 --- a/br/tests/lightning_realloc_id/data/db.test-schema.sql +++ /dev/null @@ -1,4 +0,0 @@ -create table db.test( - id int auto_increment unique key, - a int primary key -); \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data/db.test.000000000.csv b/br/tests/lightning_realloc_id/data/db.test.000000000.csv deleted file mode 100644 index f2ce71fb561c5..0000000000000 --- a/br/tests/lightning_realloc_id/data/db.test.000000000.csv +++ /dev/null @@ -1,11 +0,0 @@ -a -100 -101 -102 -103 -104 -105 -106 -107 -108 -109 \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data/db.test.000000001.sql b/br/tests/lightning_realloc_id/data/db.test.000000001.sql deleted file mode 100644 index 611b5f5dbeba6..0000000000000 --- a/br/tests/lightning_realloc_id/data/db.test.000000001.sql +++ /dev/null @@ -1,11 +0,0 @@ -insert into db.test(a) values -(200), -(201), -(202), -(203), -(204), -(205), -(206), -(207), -(208), -(209); \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data1/db-schema-create.sql b/br/tests/lightning_realloc_id/data1/db-schema-create.sql deleted file mode 100644 index c88b0e3150e76..0000000000000 --- a/br/tests/lightning_realloc_id/data1/db-schema-create.sql +++ /dev/null @@ -1 +0,0 @@ -create database db; \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data1/db.test-schema.sql b/br/tests/lightning_realloc_id/data1/db.test-schema.sql deleted file mode 100644 index 0490cd81e1c2e..0000000000000 --- a/br/tests/lightning_realloc_id/data1/db.test-schema.sql +++ /dev/null @@ -1,4 +0,0 @@ -create table db.test( - id int auto_increment unique key, - a int primary key -); \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data1/db.test.000000000.csv b/br/tests/lightning_realloc_id/data1/db.test.000000000.csv deleted file mode 100644 index 70ae8fd5a20a7..0000000000000 --- a/br/tests/lightning_realloc_id/data1/db.test.000000000.csv +++ /dev/null @@ -1,11 +0,0 @@ -a -300 -301 -302 -303 -304 -305 -306 -307 -308 -309 \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data1/db.test.000000001.sql b/br/tests/lightning_realloc_id/data1/db.test.000000001.sql deleted file mode 100644 index 461cf4c3fccaf..0000000000000 --- a/br/tests/lightning_realloc_id/data1/db.test.000000001.sql +++ /dev/null @@ -1,11 +0,0 @@ -insert into db.test(a) values -(400), -(401), -(402), -(403), -(404), -(405), -(406), -(407), -(408), -(409); \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/data2/db.test.000000000.csv b/br/tests/lightning_realloc_id/data2/db.test.000000000.csv deleted file mode 100644 index 12d1d9e0bc948..0000000000000 --- a/br/tests/lightning_realloc_id/data2/db.test.000000000.csv +++ /dev/null @@ -1,11 +0,0 @@ -a -1 -2 -3 -4 -5 -6 -7 -8 -9 -10 \ No newline at end of file diff --git a/br/tests/lightning_realloc_id/run.sh b/br/tests/lightning_realloc_id/run.sh deleted file mode 100644 index eead3b2fc1f33..0000000000000 --- a/br/tests/lightning_realloc_id/run.sh +++ /dev/null @@ -1,93 +0,0 @@ -#!/bin/sh -# -# Copyright 2022 PingCAP, Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# Basic check for whether partitioned tables work. - -set -eu -check_cluster_version 4 0 0 'local backend' -LOG_FILE1="$TEST_DIR/lightning-realloc-import1.log" -LOG_FILE2="$TEST_DIR/lightning-realloc-import2.log" -LOG_FILE3="$TEST_DIR/lightning-realloc-import3.log" - -function run_lightning_expecting_fail() { - set +e - run_lightning "$@" - ERRCODE=$? - set -e - [ "$ERRCODE" != 0 ] -} - -function check_result() { - run_sql 'SHOW DATABASES;' - check_contains 'Database: db'; - run_sql 'SHOW TABLES IN db;' - check_contains 'Tables_in_db: test' - run_sql 'SELECT count(*) FROM db.test;' - check_contains 'count(*): 20' - run_sql 'SELECT * FROM db.test;' - check_contains 'id: 15' - check_contains 'id: 20' -} - -function parallel_import() { - run_lightning -d "tests/$TEST_NAME/data" \ - --sorted-kv-dir "$TEST_DIR/lightning_realloc_import.sorted1" \ - --log-file "$LOG_FILE1" \ - --config "tests/$TEST_NAME/config.toml" & - pid1="$!" - run_lightning -d "tests/$TEST_NAME/data1" \ - --sorted-kv-dir "$TEST_DIR/lightning_realloc_import.sorted2" \ - --log-file "$LOG_FILE2" \ - --config "tests/$TEST_NAME/config.toml" & - pid2="$!" - wait "$pid1" "$pid2" -} - -function overflow_import() { - run_sql 'create database if not exists db' - run_sql 'create table db.test(id int auto_increment primary key, a int)' - run_sql 'alter table db.test auto_increment=2147483640' # too few available rowID - echo "lightning stdout:" > "$TEST_DIR/sql_res.$TEST_NAME.txt" - run_lightning_expecting_fail -d "tests/$TEST_NAME/data2" \ - --sorted-kv-dir "$TEST_DIR/lightning_realloc_import.sorted3" \ - --log-file "$LOG_FILE3" \ - --config "tests/$TEST_NAME/config2.toml" 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" - if ! grep -q "out of range" "$TEST_DIR/sql_res.$TEST_NAME.txt"; then - echo "TEST FAILED: OUTPUT DOES NOT CONTAIN 'out of range'" - exit 1 - fi -} - -function check_parallel_result() { - run_sql 'SHOW DATABASES;' - check_contains 'Database: db'; - run_sql 'SHOW TABLES IN db;' - check_contains 'Tables_in_db: test' - run_sql 'SELECT count(*) FROM db.test;' - check_contains 'count(*): 40' -} - -run_sql 'DROP DATABASE IF EXISTS db;' -export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/mydump/MockInaccurateRowID=return(true)' -run_lightning --config "tests/$TEST_NAME/config1.toml" -check_result -run_sql 'DROP DATABASE IF EXISTS db;' -parallel_import -check_parallel_result -run_sql 'DROP DATABASE IF EXISTS db;' -overflow_import -run_sql 'DROP DATABASE IF EXISTS db;' -unset GO_FAILPOINTS \ No newline at end of file From 4538cf51ad9aa18659704eb8fcc6b6ec8ee0068a Mon Sep 17 00:00:00 2001 From: Chengpeng Yan <41809508+Reminiscent@users.noreply.github.com> Date: Wed, 29 Jun 2022 17:00:39 +0800 Subject: [PATCH 26/28] planner: support the leading hint for outer join (#35264) ref pingcap/tidb#29932 --- planner/core/rule_join_reorder.go | 24 +- planner/core/rule_join_reorder_test.go | 42 +- .../core/testdata/join_reorder_suite_in.json | 61 + .../core/testdata/join_reorder_suite_out.json | 1384 ++++++++++++++++- 4 files changed, 1417 insertions(+), 94 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 0294da8bac852..02d04cd77479a 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -194,19 +194,15 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP } if leadingHintInfo != nil && leadingHintInfo.leadingJoinOrder != nil { - if hasOuterJoin { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable when we have outer join")) - } else { - if useGreedy { - ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, leadingHintInfo) - if !ok { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) - } else { - curJoinGroup = leftJoinGroup - } + if useGreedy { + ok, leftJoinGroup := baseGroupSolver.generateLeadingJoinGroup(curJoinGroup, leadingHintInfo, hasOuterJoin) + if !ok { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable, check if the leading hint table is valid")) } else { - ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable for the DP join reorder algorithm")) + curJoinGroup = leftJoinGroup } + } else { + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("leading hint is inapplicable for the DP join reorder algorithm")) } } @@ -298,7 +294,7 @@ type baseSingleGroupJoinOrderSolver struct { leadingJoinGroup LogicalPlan } -func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup []LogicalPlan, hintInfo *tableHintInfo) (bool, []LogicalPlan) { +func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup []LogicalPlan, hintInfo *tableHintInfo, hasOuterJoin bool) (bool, []LogicalPlan) { var leadingJoinGroup []LogicalPlan leftJoinGroup := make([]LogicalPlan, len(curJoinGroup)) copy(leftJoinGroup, curJoinGroup) @@ -324,6 +320,10 @@ func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup [ var usedEdges []*expression.ScalarFunction var joinType JoinType leadingJoin, leadingJoinGroup[0], usedEdges, joinType = s.checkConnection(leadingJoin, leadingJoinGroup[0]) + if hasOuterJoin && usedEdges == nil { + // If the joinGroups contain the outer join, we disable the cartesian product. + return false, nil + } leadingJoin, s.otherConds = s.makeJoin(leadingJoin, leadingJoinGroup[0], usedEdges, joinType) leadingJoinGroup = leadingJoinGroup[1:] } diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 59a53c41adb04..4c15c6052eafd 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -79,22 +79,6 @@ func TestLeadingJoinHint(t *testing.T) { tk.MustExec("create table t8(a int, b int, key(a));") runJoinReorderTestData(t, tk, "TestLeadingJoinHint") - // test cases for outer join - tk.MustExec("select /*+ leading(t1, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - tk.MustExec("select /*+ leading(t2) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - tk.MustExec("select /*+ leading(t2, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - tk.MustExec("select /*+ leading(t1, t2, t3) */ * from t1 left join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - tk.MustExec("select /*+ leading(t1, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - tk.MustExec("select /*+ leading(t1, t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - tk.MustExec("select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - // test cases for multiple leading hints tk.MustExec("select /*+ leading(t1) leading(t2) */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid")) @@ -166,13 +150,6 @@ func TestJoinOrderHint(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 There are no matching table names for (t1) in optimizer hint /*+ LEADING(t2, t1) */. Maybe you can use the table alias name", "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid")) - // conflict between table names - tk.MustExec("select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - - tk.MustExec("select /*+ leading(t1, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 leading hint is inapplicable when we have outer join")) - // table name in leading hint cross query block // Todo: Can not handle this case yet. Because when we extract the join group, it will get the join group {t1, t2, t3}. // So the table 't4' can not be used. @@ -329,3 +306,22 @@ func TestJoinOrderHint4Subquery(t *testing.T) { runJoinReorderTestData(t, tk, "TestJoinOrderHint4Subquery") } + +func TestLeadingJoinHint4OuterJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") + tk.MustExec("create table t(a int, b int, key(a));") + tk.MustExec("create table t1(a int, b int, key(a));") + tk.MustExec("create table t2(a int, b int, key(a));") + tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("create table t4(a int, b int, key(a));") + tk.MustExec("create table t5(a int, b int, key(a));") + tk.MustExec("create table t6(a int, b int, key(a));") + tk.MustExec("create table t7(a int, b int, key(a));") + tk.MustExec("create table t8(a int, b int, key(a));") + runJoinReorderTestData(t, tk, "TestLeadingJoinHint4OuterJoin") +} diff --git a/planner/core/testdata/join_reorder_suite_in.json b/planner/core/testdata/join_reorder_suite_in.json index 8b0e77e742422..3145779cc69bf 100644 --- a/planner/core/testdata/join_reorder_suite_in.json +++ b/planner/core/testdata/join_reorder_suite_in.json @@ -298,5 +298,66 @@ "select /*+ leading(t1, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;", "select /*+ leading(t3, t2@sel_2) */ t1.a, (select min(t2.a) from t2) from t1 join t3 on t1.a = t3.a;" ] + }, + { + "name": "TestLeadingJoinHint4OuterJoin", + "cases": [ + "select /*+ leading(t3, t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t3, t1) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t1, t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t3) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t1) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t3, t2, t1) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t1, t2, t3) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t3, t1, t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "select /*+ leading(t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t3, t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t1, t3, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t2, t3, t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "select /*+ leading(t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t3, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1, t2, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t3, t1, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "select /*+ leading(t1) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4, t1) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4, t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t3, t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t3, t4) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t2, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t4, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t4, t2, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1, t2, t3, t4) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "select /*+ leading(t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t2) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t3) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t4) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t2, t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t2, t3) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t4, t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t3, t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select * from ((select /*+ leading(t5) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select * from ((select /*+ leading(t6) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select * from ((select /*+ leading(t5, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select * from ((select /*+ leading(t6, t8, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t3) */ * from ((select /*+ leading(t5) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t6, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t5, t7, t8) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;" + ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 4fa0ac13bc667..f82b504714a2d 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -2865,78 +2865,74 @@ { "SQL": "select /*+ leading(t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "Plan": [ - "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], - "Warning": [ - "Warning 1815 leading hint is inapplicable when we have outer join" - ] + "Warning": null }, { "SQL": "select /*+ leading(t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "Plan": [ - "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], - "Warning": [ - "Warning 1815 leading hint is inapplicable when we have outer join" - ] + "Warning": null }, { "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "Plan": [ - "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], - "Warning": [ - "Warning 1815 leading hint is inapplicable when we have outer join" - ] + "Warning": null }, { "SQL": "select /*+ leading(t3, t2) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", "Plan": [ - "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "Projection 15609.38 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t3.a, test.t3.b", + "└─HashJoin 15609.38 root inner join, equal:[eq(test.t2.a, test.t1.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], - "Warning": [ - "Warning 1815 leading hint is inapplicable when we have outer join" - ] + "Warning": null }, { "SQL": "select /*+ leading(t3, t1) */ * from t1 join t2 on t1.a=t2.a left join t3 on t2.b=t3.b", @@ -2954,7 +2950,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ - "Warning 1815 leading hint is inapplicable when we have outer join" + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, { @@ -2989,9 +2985,7 @@ " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], - "Warning": [ - "Warning 1815 leading hint is inapplicable when we have outer join" - ] + "Warning": null }, { "SQL": "select /*+ leading(t2, t3) */ * from t1 join t2 on t1.a=t2.a right join t3 on t2.b=t3.b", @@ -3010,7 +3004,7 @@ ], "Warning": [ "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", - "Warning 1815 leading hint is inapplicable when we have outer join" + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, { @@ -3030,7 +3024,7 @@ ], "Warning": [ "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", - "Warning 1815 leading hint is inapplicable when we have outer join" + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, { @@ -3050,7 +3044,7 @@ ], "Warning": [ "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", - "Warning 1815 leading hint is inapplicable when we have outer join" + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" ] }, { @@ -4880,5 +4874,1277 @@ ] } ] + }, + { + "Name": "TestLeadingJoinHint4OuterJoin", + "Cases": [ + { + "SQL": "select /*+ leading(t3, t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t2, t1) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t2, t3) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3, t1, t2) */ * from t2 left join t1 on t2.a=t1.a left join t3 on t1.b=t3.b;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t1.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t2, t3, t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=1;", + "Plan": [ + "HashJoin 124875000.00 root CARTESIAN left outer join, left cond:[eq(test.t2.a, 1)]", + "├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─Projection(Probe) 12487.50 root test.t1.a, test.t1.b, test.t3.a, test.t3.b", + " └─HashJoin 12487.50 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1, t2) */ * from t2 left join (t1 left join t3 on t1.a=t3.a) on t2.a=t3.a;", + "Plan": [ + "HashJoin 15609.38 root left outer join, equal:[eq(test.t2.a, test.t3.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t4) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─Projection(Build) 12487.50 root test.t3.a, test.t3.b, test.t4.a, test.t4.b", + "│ └─HashJoin 12487.50 root inner join, equal:[eq(test.t4.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4, t1) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4, t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t2) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t4) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t1, t2, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t4, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4, t2, t3) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1, t2, t3, t4) */ * from (t1 left join t2 on t1.a=t2.a) left join (t3 left join t4 on t3.a=t4.a) on t2.a=t4.a;", + "Plan": [ + "HashJoin 19511.72 root left outer join, equal:[eq(test.t2.a, test.t4.a)]", + "├─HashJoin(Build) 12487.50 root inner join, equal:[eq(test.t3.a, test.t4.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t3) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select /*+ leading(t2, t3) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t4, t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from ((select t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select * from ((select /*+ leading(t5) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t6.a)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from ((select /*+ leading(t6) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": null + }, + { + "SQL": "select * from ((select /*+ leading(t5, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select * from ((select /*+ leading(t6, t8, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid" + ] + }, + { + "SQL": "select /*+ leading(t3) */ * from ((select /*+ leading(t5) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1) */ * from ((select /*+ leading(t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(t3, t1, t2) */ * from ((select /*+ leading(t6, t7) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "select /*+ leading(t3, t4) */ * from ((select /*+ leading(t5, t7, t8) */ t8.a, t8.b from t8, t7, t6, t5 where t5.a = t6.a and t6.b=t7.b) t3 left join t4 on t3.a=t4.a) left join (t1 left join t2 on t1.a=t2.a) on t1.a=t4.a;", + "Plan": [ + "HashJoin 304261169.13 root CARTESIAN inner join", + "├─HashJoin(Build) 15593.77 root inner join, equal:[eq(test.t6.a, test.t5.a)]", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "└─HashJoin(Probe) 19511.72 root left outer join, equal:[eq(test.t4.a, test.t1.a)]", + " ├─HashJoin(Build) 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 root left outer join, equal:[eq(test.t8.a, test.t4.a)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ], + "Warning": [ + "Warning 1815 We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + } + ] } ] From 5eac3b699910f650771f195b233da5674246a49d Mon Sep 17 00:00:00 2001 From: 3pointer Date: Wed, 29 Jun 2022 17:20:40 +0800 Subject: [PATCH 27/28] log-backup: check timezone when using datatime format ts string (#35811) ref pingcap/tidb#31975 --- br/pkg/task/backup.go | 10 ++++++++-- br/pkg/task/backup_test.go | 27 +++++++++++++++++++++++---- br/pkg/task/restore.go | 4 ++-- br/pkg/task/stream.go | 6 +++--- 4 files changed, 36 insertions(+), 11 deletions(-) diff --git a/br/pkg/task/backup.go b/br/pkg/task/backup.go index de94b2d385ad5..311f57e9de020 100644 --- a/br/pkg/task/backup.go +++ b/br/pkg/task/backup.go @@ -137,7 +137,7 @@ func (cfg *BackupConfig) ParseFromFlags(flags *pflag.FlagSet) error { if err != nil { return errors.Trace(err) } - cfg.BackupTS, err = ParseTSString(backupTS) + cfg.BackupTS, err = ParseTSString(backupTS, false) if err != nil { return errors.Trace(err) } @@ -528,7 +528,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig } // ParseTSString port from tidb setSnapshotTS. -func ParseTSString(ts string) (uint64, error) { +func ParseTSString(ts string, tzCheck bool) (uint64, error) { if len(ts) == 0 { return 0, nil } @@ -540,6 +540,12 @@ func ParseTSString(ts string) (uint64, error) { sc := &stmtctx.StatementContext{ TimeZone: loc, } + if tzCheck { + tzIdx, _, _, _, _ := types.GetTimezone(ts) + if tzIdx < 0 { + return 0, errors.Errorf("must set timezone when using datetime format ts") + } + } t, err := types.ParseTime(sc, ts, mysql.TypeTimestamp, types.MaxFsp) if err != nil { return 0, errors.Trace(err) diff --git a/br/pkg/task/backup_test.go b/br/pkg/task/backup_test.go index ef3f33d27ee54..25d6de8ee66fb 100644 --- a/br/pkg/task/backup_test.go +++ b/br/pkg/task/backup_test.go @@ -16,21 +16,40 @@ func TestParseTSString(t *testing.T) { err error ) - ts, err = ParseTSString("") + ts, err = ParseTSString("", false) require.NoError(t, err) require.Zero(t, ts) - ts, err = ParseTSString("400036290571534337") + ts, err = ParseTSString("400036290571534337", false) require.NoError(t, err) require.Equal(t, uint64(400036290571534337), ts) - ts, err = ParseTSString("2021-01-01 01:42:23") + ts, err = ParseTSString("2021-01-01 01:42:23", false) require.NoError(t, err) localTime := time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, time.Local) - localTimestamp := localTime.Unix() localTSO := uint64((localTimestamp << 18) * 1000) require.Equal(t, localTSO, ts) + + _, err = ParseTSString("2021-01-01 01:42:23", true) + require.Error(t, err) + require.Regexp(t, "must set timezone*", err.Error()) + + ts, err = ParseTSString("2021-01-01 01:42:23+00:00", true) + require.NoError(t, err) + localTime = time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, time.UTC) + localTimestamp = localTime.Unix() + localTSO = uint64((localTimestamp << 18) * 1000) + require.Equal(t, localTSO, ts) + + ts, err = ParseTSString("2021-01-01 01:42:23+08:00", true) + require.NoError(t, err) + secondsEastOfUTC := int((8 * time.Hour).Seconds()) + beijing := time.FixedZone("Beijing Time", secondsEastOfUTC) + localTime = time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, beijing) + localTimestamp = localTime.Unix() + localTSO = uint64((localTimestamp << 18) * 1000) + require.Equal(t, localTSO, ts) } func TestParseCompressionType(t *testing.T) { diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 1e6cbd2432065..d139d58a12186 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -182,14 +182,14 @@ func (cfg *RestoreConfig) ParseStreamRestoreFlags(flags *pflag.FlagSet) error { if err != nil { return errors.Trace(err) } - if cfg.StartTS, err = ParseTSString(tsString); err != nil { + if cfg.StartTS, err = ParseTSString(tsString, true); err != nil { return errors.Trace(err) } tsString, err = flags.GetString(FlagStreamRestoreTS) if err != nil { return errors.Trace(err) } - if cfg.RestoreTS, err = ParseTSString(tsString); err != nil { + if cfg.RestoreTS, err = ParseTSString(tsString, true); err != nil { return errors.Trace(err) } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 7d8b717f07e0d..e3337cc49b039 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -189,7 +189,7 @@ func (cfg *StreamConfig) ParseStreamTruncateFromFlags(flags *pflag.FlagSet) erro if err != nil { return errors.Trace(err) } - if cfg.Until, err = ParseTSString(tsString); err != nil { + if cfg.Until, err = ParseTSString(tsString, true); err != nil { return errors.Trace(err) } if cfg.SkipPrompt, err = flags.GetBool(flagYes); err != nil { @@ -213,7 +213,7 @@ func (cfg *StreamConfig) ParseStreamStartFromFlags(flags *pflag.FlagSet) error { return errors.Trace(err) } - if cfg.StartTS, err = ParseTSString(tsString); err != nil { + if cfg.StartTS, err = ParseTSString(tsString, true); err != nil { return errors.Trace(err) } @@ -222,7 +222,7 @@ func (cfg *StreamConfig) ParseStreamStartFromFlags(flags *pflag.FlagSet) error { return errors.Trace(err) } - if cfg.EndTS, err = ParseTSString(tsString); err != nil { + if cfg.EndTS, err = ParseTSString(tsString, true); err != nil { return errors.Trace(err) } From 47a4f3a017768dda0b9a826451486c5539bc8427 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 29 Jun 2022 20:16:40 +0800 Subject: [PATCH 28/28] *: enable unconvert (#35821) ref pingcap/tidb#35345 --- DEPS.bzl | 7 ++++ br/pkg/lightning/mydump/BUILD.bazel | 1 - build/BUILD.bazel | 1 + build/linter/unconvert/BUILD.bazel | 14 +++++++ build/linter/unconvert/analysis.go | 61 +++++++++++++++++++++++++++++ build/linter/util/BUILD.bazel | 1 + build/linter/util/util.go | 20 ++++++++++ build/nogo_config.json | 9 +++++ executor/seqtest/BUILD.bazel | 1 + go.mod | 3 ++ go.sum | 3 ++ sessiontxn/isolation/BUILD.bazel | 2 + testkit/testfork/BUILD.bazel | 1 + 13 files changed, 123 insertions(+), 1 deletion(-) create mode 100644 build/linter/unconvert/BUILD.bazel create mode 100644 build/linter/unconvert/analysis.go diff --git a/DEPS.bzl b/DEPS.bzl index 65f9d5b4b657c..0bcdc6e81964a 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -942,6 +942,13 @@ def go_deps(): sum = "h1:leSNB7iYzLYSSx3J/s5sVf4Drkc68W2wm4Ixh/mr0us=", version = "v0.0.0-20180630174525-215b22d4de21", ) + go_repository( + name = "com_github_golangci_unconvert", + build_file_proto_mode = "disable", + importpath = "github.com/golangci/unconvert", + sum = "h1:zwtduBRr5SSWhqsYNgcuWO2kFlpdOZbP0+yRjmvPGys=", + version = "v0.0.0-20180507085042-28b1c447d1f4", + ) go_repository( name = "com_github_gomodule_redigo", diff --git a/br/pkg/lightning/mydump/BUILD.bazel b/br/pkg/lightning/mydump/BUILD.bazel index 24d3545571ac2..1736f87af679f 100644 --- a/br/pkg/lightning/mydump/BUILD.bazel +++ b/br/pkg/lightning/mydump/BUILD.bazel @@ -31,7 +31,6 @@ go_library( "//util/slice", "//util/table-filter", "@com_github_pingcap_errors//:errors", - "@com_github_pingcap_failpoint//:failpoint", "@com_github_xitongsys_parquet_go//parquet", "@com_github_xitongsys_parquet_go//reader", "@com_github_xitongsys_parquet_go//source", diff --git a/build/BUILD.bazel b/build/BUILD.bazel index b283c4b2e0f20..cddd70f8d5876 100644 --- a/build/BUILD.bazel +++ b/build/BUILD.bazel @@ -110,5 +110,6 @@ nogo( "//build/linter/gofmt:gofmt", "//build/linter/ineffassign:ineffassign", "//build/linter/prealloc:prealloc", + "//build/linter/unconvert:unconvert", ] + staticcheck_analyzers(STATICHECK_ANALYZERS), ) diff --git a/build/linter/unconvert/BUILD.bazel b/build/linter/unconvert/BUILD.bazel new file mode 100644 index 0000000000000..23ececa09cd70 --- /dev/null +++ b/build/linter/unconvert/BUILD.bazel @@ -0,0 +1,14 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "unconvert", + srcs = ["analysis.go"], + importpath = "github.com/pingcap/tidb/build/linter/unconvert", + visibility = ["//visibility:public"], + deps = [ + "//build/linter/util", + "@com_github_golangci_unconvert//:unconvert", + "@org_golang_x_tools//go/analysis", + "@org_golang_x_tools//go/loader", + ], +) diff --git a/build/linter/unconvert/analysis.go b/build/linter/unconvert/analysis.go new file mode 100644 index 0000000000000..89da08bccd4aa --- /dev/null +++ b/build/linter/unconvert/analysis.go @@ -0,0 +1,61 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package unconvert + +import ( + "fmt" + "go/token" + "go/types" + + unconvertAPI "github.com/golangci/unconvert" + "github.com/pingcap/tidb/build/linter/util" + "golang.org/x/tools/go/analysis" + "golang.org/x/tools/go/loader" +) + +// Name is the name of the analyzer. +const Name = "unconvert" + +// Analyzer is the analyzer struct of unconvert. +var Analyzer = &analysis.Analyzer{ + Name: Name, + Doc: "Remove unnecessary type conversions", + Run: run, +} + +func run(pass *analysis.Pass) (interface{}, error) { + var createdPkgs []*loader.PackageInfo + createdPkgs = append(createdPkgs, util.MakeFakeLoaderPackageInfo(pass)) + allPkgs := map[*types.Package]*loader.PackageInfo{} + for _, pkg := range createdPkgs { + pkg := pkg + allPkgs[pkg.Pkg] = pkg + } + prog := &loader.Program{ + Fset: pass.Fset, + Imported: nil, // not used without .Created in any linter + Created: createdPkgs, // all initial packages + AllPackages: allPkgs, // all initial packages and their depndencies + } + positions := unconvertAPI.Run(prog) + if len(positions) == 0 { + return nil, nil + } + + for _, pos := range positions { + pass.Reportf(token.Pos(pos.Offset), fmt.Sprintf("[%s] Unnecessary conversion", Name)) + } + return nil, nil +} diff --git a/build/linter/util/BUILD.bazel b/build/linter/util/BUILD.bazel index 4ac3fec064d07..f8e81695c03eb 100644 --- a/build/linter/util/BUILD.bazel +++ b/build/linter/util/BUILD.bazel @@ -8,5 +8,6 @@ go_library( deps = [ "@co_honnef_go_tools//analysis/report", "@org_golang_x_tools//go/analysis", + "@org_golang_x_tools//go/loader", ], ) diff --git a/build/linter/util/util.go b/build/linter/util/util.go index d476173a973a0..5b58883576d60 100644 --- a/build/linter/util/util.go +++ b/build/linter/util/util.go @@ -22,6 +22,7 @@ import ( "strings" "golang.org/x/tools/go/analysis" + "golang.org/x/tools/go/loader" "honnef.co/go/tools/analysis/report" ) @@ -147,3 +148,22 @@ func FormatCode(code string) string { return fmt.Sprintf("`%s`", code) } + +// MakeFakeLoaderPackageInfo creates a fake loader.PackageInfo for a given package. +func MakeFakeLoaderPackageInfo(pass *analysis.Pass) *loader.PackageInfo { + var errs []error + + typeInfo := pass.TypesInfo + + return &loader.PackageInfo{ + Pkg: pass.Pkg, + Importable: true, // not used + TransitivelyErrorFree: true, // not used + + // use compiled (preprocessed) go files AST; + // AST linters use not preprocessed go files AST + Files: pass.Files, + Errors: errs, + Info: *typeInfo, + } +} diff --git a/build/nogo_config.json b/build/nogo_config.json index 8f7f4e940a597..5200c55164431 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -210,6 +210,15 @@ ".*_generated\\.go$": "ignore generated code" } }, + "unconvert": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*\\.pb\\.go$": "generated code", + "parser/parser.go": "generated code", + "/cgo/": "no need to vet third party code for cgo", + ".*_generated\\.go$": "ignore generated code" + } + }, "unmarshal": { "exclude_files": { "/external/": "no need to vet third party code", diff --git a/executor/seqtest/BUILD.bazel b/executor/seqtest/BUILD.bazel index 3f0a4bfd1291d..c248e2e1fd30a 100644 --- a/executor/seqtest/BUILD.bazel +++ b/executor/seqtest/BUILD.bazel @@ -17,6 +17,7 @@ go_test( "//kv", "//meta/autoid", "//metrics", + "//parser/ast", "//parser/model", "//parser/mysql", "//parser/terror", diff --git a/go.mod b/go.mod index 8943a2403025a..443d6ee0d5431 100644 --- a/go.mod +++ b/go.mod @@ -99,11 +99,14 @@ require ( github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 github.com/charithe/durationcheck v0.0.9 github.com/golangci/gofmt v0.0.0-20190930125516-244bba706f1a + github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4 github.com/gordonklaus/ineffassign v0.0.0-20210914165742-4cc7213b9bc8 github.com/kyoh86/exportloopref v0.1.8 honnef.co/go/tools v0.0.1-2020.1.4 ) +require github.com/kisielk/gotool v1.0.0 // indirect + require ( cloud.google.com/go v0.100.2 // indirect cloud.google.com/go/compute v1.2.0 // indirect diff --git a/go.sum b/go.sum index 2dcc2c3e4cfeb..395e388d04d89 100644 --- a/go.sum +++ b/go.sum @@ -347,6 +347,8 @@ github.com/golangci/gofmt v0.0.0-20190930125516-244bba706f1a h1:iR3fYXUjHCR97qWS github.com/golangci/gofmt v0.0.0-20190930125516-244bba706f1a/go.mod h1:9qCChq59u/eW8im404Q2WWTrnBUQKjpNYKMbU4M7EFU= github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21 h1:leSNB7iYzLYSSx3J/s5sVf4Drkc68W2wm4Ixh/mr0us= github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21/go.mod h1:tf5+bzsHdTM0bsB7+8mt0GUMvjCgwLpTapNZHU8AajI= +github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4 h1:zwtduBRr5SSWhqsYNgcuWO2kFlpdOZbP0+yRjmvPGys= +github.com/golangci/unconvert v0.0.0-20180507085042-28b1c447d1f4/go.mod h1:Izgrg8RkN3rCIMLGE9CyYmU9pY2Jer6DgANEnZ/L/cQ= github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -511,6 +513,7 @@ github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiD github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= diff --git a/sessiontxn/isolation/BUILD.bazel b/sessiontxn/isolation/BUILD.bazel index 1345c09f08b21..16ebeaecfb9a0 100644 --- a/sessiontxn/isolation/BUILD.bazel +++ b/sessiontxn/isolation/BUILD.bazel @@ -48,10 +48,12 @@ go_test( "//parser", "//parser/ast", "//planner", + "//session", "//sessionctx", "//sessiontxn", "//testkit", "//testkit/testsetup", + "//types", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/kvrpcpb", diff --git a/testkit/testfork/BUILD.bazel b/testkit/testfork/BUILD.bazel index 743bd70da5b0d..89d1e33ac65a5 100644 --- a/testkit/testfork/BUILD.bazel +++ b/testkit/testfork/BUILD.bazel @@ -15,4 +15,5 @@ go_test( name = "testfork_test", srcs = ["fork_test.go"], embed = [":testfork"], + deps = ["@com_github_stretchr_testify//require"], )