From 3024769cbbde29bd09f246bf6421e72e043b458e Mon Sep 17 00:00:00 2001 From: longfangsong Date: Wed, 24 Feb 2021 13:17:01 +0800 Subject: [PATCH 01/26] Add txn state's recording table Signed-off-by: longfangsong --- executor/adapter.go | 8 +- executor/builder.go | 5 +- executor/infoschema_reader.go | 9 + infoschema/cluster.go | 3 + infoschema/tables.go | 19 +- session/session.go | 49 ++++- session/txn.go | 2 + util/txnstateRecorder/txnstate_recorder.go | 206 +++++++++++++++++++++ 8 files changed, 294 insertions(+), 7 deletions(-) create mode 100644 util/txnstateRecorder/txnstate_recorder.go diff --git a/executor/adapter.go b/executor/adapter.go index f56271de9845e..182ac1deb04d3 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -55,6 +55,8 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/txnstateRecorder" + "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -377,6 +379,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if txn.Valid() { txnStartTS = txn.StartTS() } + return &recordSet{ executor: e, stmt: a, @@ -581,7 +584,9 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { var lockKeyStats *execdetails.LockKeysDetails ctx = context.WithValue(ctx, execdetails.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() + txnstateRecorder.ReportBlocked(txn.StartTS()) err = txn.LockKeys(ctx, lockCtx, keys...) + txnstateRecorder.ReportUnblocked(txn.StartTS()) if lockKeyStats != nil { seVars.StmtCtx.MergeLockKeysExecDetails(lockKeyStats) } @@ -590,6 +595,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } e, err = a.handlePessimisticLockError(ctx, err) if err != nil { + // todo: Report deadlock if ErrDeadlock.Equal(err) { metrics.StatementDeadlockDetectDuration.Observe(time.Since(startLocking).Seconds()) } @@ -839,7 +845,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults boo } sessVars.StmtCtx.RuntimeStatsColl.RegisterStats(a.Plan.ID(), statsWithCommit) } - // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. + // `LogSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. a.LogSlowQuery(txnTS, succ, hasMoreResults) a.SummaryStmt(succ) prevStmt := a.GetTextToLog() diff --git a/executor/builder.go b/executor/builder.go index c40e36797a432..a1f8719af9d6f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1539,7 +1539,10 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TablePlacementPolicy), strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), strings.ToLower(infoschema.TableClientErrorsSummaryByUser), - strings.ToLower(infoschema.TableClientErrorsSummaryByHost): + strings.ToLower(infoschema.TableClientErrorsSummaryByHost), + strings.ToLower(infoschema.TablePlacementPolicy), + strings.ToLower(infoschema.TableTiDBTrx), + strings.ToLower(infoschema.ClusterTableTiDBTrx): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 8d5f44c785ba2..3817d868213f0 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -57,6 +57,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/txnstateRecorder" "go.etcd.io/etcd/clientv3" ) @@ -148,6 +149,9 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex infoschema.TableClientErrorsSummaryByUser, infoschema.TableClientErrorsSummaryByHost: err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) + case infoschema.TableTiDBTrx, + infoschema.ClusterTableTiDBTrx: + err = e.setDataForTiDBTrx(sctx) } if err != nil { return nil, err @@ -1977,6 +1981,11 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context return nil } +func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) error { + e.rows = txnstateRecorder.Datums() + return nil +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/infoschema/cluster.go b/infoschema/cluster.go index 2bb3998934815..78199bd7f1327 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -34,6 +34,8 @@ const ( ClusterTableStatementsSummary = "CLUSTER_STATEMENTS_SUMMARY" // ClusterTableStatementsSummaryHistory is the string constant of cluster statement summary history table. ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY" + // ClusterTableTiDBTrx is the string constant of cluster transaction running table. + ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX" ) // memTableToClusterTables means add memory table to cluster table. @@ -42,6 +44,7 @@ var memTableToClusterTables = map[string]string{ TableProcesslist: ClusterTableProcesslist, TableStatementsSummary: ClusterTableStatementsSummary, TableStatementsSummaryHistory: ClusterTableStatementsSummaryHistory, + TableTiDBTrx: ClusterTableTiDBTrx, } func init() { diff --git a/infoschema/tables.go b/infoschema/tables.go index 9d48b67cf0189..2a33e98bfa15b 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -161,6 +161,8 @@ const ( TableClientErrorsSummaryByUser = "CLIENT_ERRORS_SUMMARY_BY_USER" // TableClientErrorsSummaryByHost is the string constant of client errors table. TableClientErrorsSummaryByHost = "CLIENT_ERRORS_SUMMARY_BY_HOST" + // TableTiDBTrx is current running transaction status table. + TableTiDBTrx = "TIDB_TRX" ) var tableIDMap = map[string]int64{ @@ -229,10 +231,12 @@ var tableIDMap = map[string]int64{ TableStorageStats: autoid.InformationSchemaDBID + 63, TableTiFlashTables: autoid.InformationSchemaDBID + 64, TableTiFlashSegments: autoid.InformationSchemaDBID + 65, - TablePlacementPolicy: autoid.InformationSchemaDBID + 66, + TablePlacementPolicy: autoid.InformationSchemaDBID + 68, TableClientErrorsSummaryGlobal: autoid.InformationSchemaDBID + 67, TableClientErrorsSummaryByUser: autoid.InformationSchemaDBID + 68, - TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69, + TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 66, + TableTiDBTrx: autoid.InformationSchemaDBID + 67, + ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 69, } type columnInfo struct { @@ -1331,6 +1335,16 @@ var tableClientErrorsSummaryByHostCols = []columnInfo{ {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26}, } +var tableTiDBTrxCols = []columnInfo{ + {name: "ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, + {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, + {name: "SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, + {name: "STATE", tp: mysql.TypeLonglong, size: 64, comment: "Current running state of the transaction"}, + {name: "COMMIT_TRY_COUNT", tp: mysql.TypeLonglong, size: 64, comment: "Current transactions commit retry count"}, + {name: "COMMITSTART_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current commit's start time"}, + {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1700,6 +1714,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableClientErrorsSummaryGlobal: tableClientErrorsSummaryGlobalCols, TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols, TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols, + TableTiDBTrx: tableTiDBTrxCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/session/session.go b/session/session.go index 5c0f4ea71d4fd..3e585219af215 100644 --- a/session/session.go +++ b/session/session.go @@ -41,6 +41,9 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tipb/go-binlog" + "go.uber.org/zap" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -77,8 +80,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" - "github.com/pingcap/tipb/go-binlog" - "go.uber.org/zap" + "github.com/pingcap/tidb/util/txnstateRecorder" ) var ( @@ -502,7 +504,9 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(kv.GuaranteeLinearizability, s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } - + if s.txn.Valid() { + txnstateRecorder.ReportCommitStarted(s.txn.StartTS()) + } return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) } @@ -584,6 +588,10 @@ func (s *session) CommitTxn(ctx context.Context) error { var commitDetail *execdetails.CommitDetails ctx = context.WithValue(ctx, execdetails.CommitDetailCtxKey, &commitDetail) + startTs := uint64(0) + if s.txn.Valid() { + startTs = s.txn.StartTS() + } err := s.doCommitWithRetry(ctx) if commitDetail != nil { s.sessionVars.StmtCtx.MergeExecDetails(nil, commitDetail) @@ -595,6 +603,9 @@ func (s *session) CommitTxn(ctx context.Context) error { } }) s.sessionVars.TxnCtx.Cleanup() + if startTs != 0 { + txnstateRecorder.ReportTxnEnd(startTs) + } return err } @@ -604,7 +615,10 @@ func (s *session) RollbackTxn(ctx context.Context) { defer span1.Finish() } + startTs := uint64(0) if s.txn.Valid() { + startTs = s.txn.StartTS() + txnstateRecorder.ReportRollbackStarted(startTs) terror.Log(s.txn.Rollback()) } if ctx.Value(inCloseSession{}) == nil { @@ -613,6 +627,9 @@ func (s *session) RollbackTxn(ctx context.Context) { s.txn.changeToInvalid() s.sessionVars.TxnCtx.Cleanup() s.sessionVars.SetInTxn(false) + if startTs != 0 { + txnstateRecorder.ReportTxnEnd(startTs) + } } func (s *session) GetClient() kv.Client { @@ -813,6 +830,27 @@ func (s *session) sysSessionPool() sessionPool { return domain.GetDomain(s).SysSessionPool() } +func execRestrictedSQL(ctx context.Context, se *session, sql string) ([]chunk.Row, []*ast.ResultField, error) { + ctx = context.WithValue(ctx, execdetails.StmtExecDetailKey, &execdetails.StmtExecDetails{}) + startTime := time.Now() + rs, err := se.ExecuteInternal(ctx, sql) + if rs != nil { + defer terror.Call(rs.Close) + } + if err != nil || rs == nil { + return nil, nil, err + } + + // Execute all recordset, take out the first one as result. + rows, err := drainRecordSet(ctx, se, rs) + if err != nil { + return nil, nil, err + } + + metrics.QueryDurationHistogram.WithLabelValues(metrics.LblInternal).Observe(time.Since(startTime).Seconds()) + return rows, rs.Fields(), err +} + func createSessionFunc(store kv.Storage) pools.Factory { return func() (pools.Resource, error) { se, err := createSession(store) @@ -1376,6 +1414,11 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, } func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { + if s.txn.Valid() { + _, digest := parser.NormalizeDigest(stmtNode.Text()) + txnstateRecorder.ReportStatementStartExecute(s.txn.StartTS(), digest) + } + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.ExecuteStmt", opentracing.ChildOf(span.Context())) defer span1.Finish() diff --git a/session/txn.go b/session/txn.go index bccf130fb988a..6065bb7b0643e 100644 --- a/session/txn.go +++ b/session/txn.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/txnstateRecorder" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" ) @@ -180,6 +181,7 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t + txnstateRecorder.ReportTxnStart(txn.StartTS()) txn.initStmtBuf() return nil } diff --git a/util/txnstateRecorder/txnstate_recorder.go b/util/txnstateRecorder/txnstate_recorder.go new file mode 100644 index 0000000000000..b0f937b2f8034 --- /dev/null +++ b/util/txnstateRecorder/txnstate_recorder.go @@ -0,0 +1,206 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package txnstateRecorder is for recording the transaction running state on current tidb instance +// so we can display them in `information_schema.TIDB_TRX` +package txnstateRecorder + +import ( + "sync" + "time" + + "github.com/pingcap/log" + "go.uber.org/zap" + + "github.com/pingcap/parser/mysql" + + "github.com/pingcap/tidb/types" +) + +// TxnRunningState is the current state of a transaction +type TxnRunningState = int + +const ( + // TxnRunningNormal means the transaction is running normally + TxnRunningNormal TxnRunningState = iota + // TxnLockWaiting means the transaction is blocked on a lock + TxnLockWaiting TxnRunningState = iota + // TxnCommitting means the transaction is (at least trying to) committing + TxnCommitting TxnRunningState = iota + // TxnRollingBack means the transaction is rolling back + TxnRollingBack TxnRunningState = iota +) + +// txnStateEntry is an entry to be stored in `information_schema.TIDB_TRX` +type txnStateEntry struct { + // start timestamp of the transaction + // also worked as transaction id + startTs uint64 + // todo: Shall we parse startTs to get it? + // Pros: Save memory, some how is the "global" timestamp in a cluster(so for the CLUSTER_TIDB_TRX, this field would be more useful) + // Cons: May different with result of "NOW()" + humanReadableStartTime time.Time + // digest of SQL current running + currentSQLDigest string + // current executing state + state TxnRunningState + // how many times the transaction tries to commit + commitCount uint64 + // last trying of commit start time, nil if commitCount is 0 + commitStartTime *time.Time + // last trying to block start time + // todo: currently even if stmtState is not Blocking, blockStartTime is not nil (showing last block), is it the preferred behaviour? + blockStartTime *time.Time +} + +// storage place for `information_schema.TIDB_TRX` +// todo: is it necessary to port executor/concurrent_map.go here? +type stateMap struct { + sync.Mutex + items map[uint64]*txnStateEntry +} + +var stateStorage = stateMap{ + items: map[uint64]*txnStateEntry{}, +} + +func (e *txnStateEntry) onStatementStartExecute(sqlDigest string) { + e.currentSQLDigest = sqlDigest +} + +func (e *txnStateEntry) onRollbackStarted() { + e.state = TxnRollingBack +} + +func (e *txnStateEntry) onCommitStarted() { + e.state = TxnCommitting + now := time.Now() + e.commitStartTime = &now + e.commitCount++ +} + +func (e *txnStateEntry) onBlocked() { + e.state = TxnLockWaiting + now := time.Now() + e.blockStartTime = &now +} + +func (e *txnStateEntry) onUnblocked() { + e.state = TxnRunningNormal +} + +func (e *txnStateEntry) toDatum() []types.Datum { + var commitStartTime interface{} + if e.commitStartTime == nil { + commitStartTime = nil + } else { + commitStartTime = types.NewTime(types.FromGoTime(*e.commitStartTime), mysql.TypeTimestamp, 0) + } + var blockStartTime interface{} + if e.blockStartTime == nil { + blockStartTime = nil + } else { + blockStartTime = types.NewTime(types.FromGoTime(*e.blockStartTime), mysql.TypeTimestamp, 0) + } + return types.MakeDatums( + e.startTs, + types.NewTime(types.FromGoTime(e.humanReadableStartTime), mysql.TypeTimestamp, 0), + e.currentSQLDigest, + e.state, + e.commitCount, + commitStartTime, + blockStartTime) +} + +// ReportTxnStart is expected to be called when a transaction starts +func ReportTxnStart(txnID uint64) { + stateStorage.Lock() + defer stateStorage.Unlock() + stateStorage.items[txnID] = &txnStateEntry{ + startTs: txnID, + humanReadableStartTime: time.Now(), + currentSQLDigest: "", + state: TxnRunningNormal, + commitCount: 0, + commitStartTime: nil, + blockStartTime: nil, + } +} + +// ReportStatementStartExecute is expected to be called when a statement starts to run in a transaction +func ReportStatementStartExecute(txnID uint64, sqlDigest string) { + stateStorage.Lock() + defer stateStorage.Unlock() + if item, ok := stateStorage.items[txnID]; ok { + item.onStatementStartExecute(sqlDigest) + } +} + +// ReportRollbackStarted is expected to be called when a transaction starts to rollback +func ReportRollbackStarted(txnID uint64) { + stateStorage.Lock() + defer stateStorage.Unlock() + if item, ok := stateStorage.items[txnID]; ok { + item.onRollbackStarted() + } +} + +// ReportCommitStarted is expected to be called when a transaction starts to commit (call once for each try) +func ReportCommitStarted(txnID uint64) { + stateStorage.Lock() + defer stateStorage.Unlock() + if item, ok := stateStorage.items[txnID]; ok { + item.onCommitStarted() + } +} + +// ReportBlocked is expected to be called when a transaction is blocked when trying to acquiring a pessimistic lock +func ReportBlocked(txnID uint64) { + stateStorage.Lock() + defer stateStorage.Unlock() + if item, ok := stateStorage.items[txnID]; ok { + item.onBlocked() + } +} + +// ReportUnblocked is expected to be called when a transaction is unblocked +func ReportUnblocked(txnID uint64) { + stateStorage.Lock() + defer stateStorage.Unlock() + if item, ok := stateStorage.items[txnID]; ok { + item.onUnblocked() + } +} + +// ReportTxnEnd is expected to be called when a transaction end +func ReportTxnEnd(txnID uint64) { + stateStorage.Lock() + defer stateStorage.Unlock() + if _, ok := stateStorage.items[txnID]; ok { + delete(stateStorage.items, txnID) + } +} + +// Datums is used to read all the txn states to Datums, for querying by SQL +func Datums() [][]types.Datum { + stateStorage.Lock() + defer stateStorage.Unlock() + var result [][]types.Datum + for _, status := range stateStorage.items { + log.Info("stateStorage", zap.Any("item", status)) + if status != nil { + result = append(result, status.toDatum()) + } + } + return result +} From 3bea616ab27400b5faf77c49b78a7a070fb1862b Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 18 Mar 2021 13:39:41 +0800 Subject: [PATCH 02/26] Rebase Signed-off-by: longfangsong --- infoschema/tables.go | 9 +++++---- util/txnstateRecorder/txnstate_recorder_test.go | 16 ++++++++++++++++ 2 files changed, 21 insertions(+), 4 deletions(-) create mode 100644 util/txnstateRecorder/txnstate_recorder_test.go diff --git a/infoschema/tables.go b/infoschema/tables.go index 2a33e98bfa15b..53fa7a888d355 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain/infosync" @@ -231,12 +232,12 @@ var tableIDMap = map[string]int64{ TableStorageStats: autoid.InformationSchemaDBID + 63, TableTiFlashTables: autoid.InformationSchemaDBID + 64, TableTiFlashSegments: autoid.InformationSchemaDBID + 65, - TablePlacementPolicy: autoid.InformationSchemaDBID + 68, + TablePlacementPolicy: autoid.InformationSchemaDBID + 66, TableClientErrorsSummaryGlobal: autoid.InformationSchemaDBID + 67, TableClientErrorsSummaryByUser: autoid.InformationSchemaDBID + 68, - TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 66, - TableTiDBTrx: autoid.InformationSchemaDBID + 67, - ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 69, + TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69, + TableTiDBTrx: autoid.InformationSchemaDBID + 70, + ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 71, } type columnInfo struct { diff --git a/util/txnstateRecorder/txnstate_recorder_test.go b/util/txnstateRecorder/txnstate_recorder_test.go new file mode 100644 index 0000000000000..633bfd4da7367 --- /dev/null +++ b/util/txnstateRecorder/txnstate_recorder_test.go @@ -0,0 +1,16 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package txnstateRecorder is for recording the transaction running state on current tidb instance +// so we can display them in `information_schema.TIDB_TRX` +package txnstateRecorder From 1f2467d19e8d1db1170ea6392c4199e00020d369 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 18 Mar 2021 15:52:49 +0800 Subject: [PATCH 03/26] Basic unit test Signed-off-by: longfangsong --- .../txnstate_recorder_test.go | 55 ++++++++++++++++++- 1 file changed, 54 insertions(+), 1 deletion(-) diff --git a/util/txnstateRecorder/txnstate_recorder_test.go b/util/txnstateRecorder/txnstate_recorder_test.go index 633bfd4da7367..9e7fd678ae714 100644 --- a/util/txnstateRecorder/txnstate_recorder_test.go +++ b/util/txnstateRecorder/txnstate_recorder_test.go @@ -13,4 +13,57 @@ // Package txnstateRecorder is for recording the transaction running state on current tidb instance // so we can display them in `information_schema.TIDB_TRX` -package txnstateRecorder +package txnstateRecorder_test + +import ( + "testing" + + . "github.com/pingcap/check" + + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/testkit" +) + +type testTxnStateRecorderSuite struct { + store kv.Storage +} + +var _ = Suite(&testTxnStateRecorderSuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +func (s *testTxnStateRecorderSuite) SetUpSuite(c *C) { + var err error + s.store, err = mockstore.NewMockStore() + if err != nil { + panic(err) + } + _, err = session.BootstrapSession(s.store) + if err != nil { + panic(err) + } +} + +func (s *testTxnStateRecorderSuite) TearDownSuite(c *C) { + _ = s.store.Close() +} + +func (s *testTxnStateRecorderSuite) TestBasicRecordingTxnState(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("create table t(id int, a varchar(128));") + tk.MustExec("insert into t(id, a) values (1, 'abcd');") + rows := tk.MustQuery("select * from information_schema.TIDB_TRX;").Rows() + c.Assert(len(rows), Equals, 0) + tk.MustExec("BEGIN;") + tk.MustExec("select * from t for update;") + rows = tk.MustQuery("select * from information_schema.TIDB_TRX;").Rows() + c.Assert(len(rows), Equals, 1) + tk.MustExec("COMMIT;") + rows = tk.MustQuery("select * from information_schema.TIDB_TRX;").Rows() + c.Assert(len(rows), Equals, 0) +} From 062c95c0d22ac2dc5e10f6075e448c0dadb9b70d Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 22 Apr 2021 13:40:39 +0800 Subject: [PATCH 04/26] Remove unnecessary columns --- infoschema/tables.go | 2 -- util/txnstateRecorder/txnstate_recorder.go | 21 +-------------------- 2 files changed, 1 insertion(+), 22 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index d1f369ac2ff41..554a72afc303a 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1342,8 +1342,6 @@ var tableTiDBTrxCols = []columnInfo{ {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, {name: "SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeLonglong, size: 64, comment: "Current running state of the transaction"}, - {name: "COMMIT_TRY_COUNT", tp: mysql.TypeLonglong, size: 64, comment: "Current transactions commit retry count"}, - {name: "COMMITSTART_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current commit's start time"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, } diff --git a/util/txnstateRecorder/txnstate_recorder.go b/util/txnstateRecorder/txnstate_recorder.go index b0f937b2f8034..8b60b051fe01f 100644 --- a/util/txnstateRecorder/txnstate_recorder.go +++ b/util/txnstateRecorder/txnstate_recorder.go @@ -20,11 +20,9 @@ import ( "time" "github.com/pingcap/log" - "go.uber.org/zap" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/types" + "go.uber.org/zap" ) // TxnRunningState is the current state of a transaction @@ -54,10 +52,6 @@ type txnStateEntry struct { currentSQLDigest string // current executing state state TxnRunningState - // how many times the transaction tries to commit - commitCount uint64 - // last trying of commit start time, nil if commitCount is 0 - commitStartTime *time.Time // last trying to block start time // todo: currently even if stmtState is not Blocking, blockStartTime is not nil (showing last block), is it the preferred behaviour? blockStartTime *time.Time @@ -84,9 +78,6 @@ func (e *txnStateEntry) onRollbackStarted() { func (e *txnStateEntry) onCommitStarted() { e.state = TxnCommitting - now := time.Now() - e.commitStartTime = &now - e.commitCount++ } func (e *txnStateEntry) onBlocked() { @@ -100,12 +91,6 @@ func (e *txnStateEntry) onUnblocked() { } func (e *txnStateEntry) toDatum() []types.Datum { - var commitStartTime interface{} - if e.commitStartTime == nil { - commitStartTime = nil - } else { - commitStartTime = types.NewTime(types.FromGoTime(*e.commitStartTime), mysql.TypeTimestamp, 0) - } var blockStartTime interface{} if e.blockStartTime == nil { blockStartTime = nil @@ -117,8 +102,6 @@ func (e *txnStateEntry) toDatum() []types.Datum { types.NewTime(types.FromGoTime(e.humanReadableStartTime), mysql.TypeTimestamp, 0), e.currentSQLDigest, e.state, - e.commitCount, - commitStartTime, blockStartTime) } @@ -131,8 +114,6 @@ func ReportTxnStart(txnID uint64) { humanReadableStartTime: time.Now(), currentSQLDigest: "", state: TxnRunningNormal, - commitCount: 0, - commitStartTime: nil, blockStartTime: nil, } } From 1dd37009e317edd93392bf81e0b3131418728339 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 22 Apr 2021 16:22:34 +0800 Subject: [PATCH 05/26] move some report code to TxnState Signed-off-by: longfangsong --- executor/adapter.go | 3 --- session/session.go | 3 --- session/txn.go | 9 +++++++++ 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 69d09fd8d62df..e2d82c0c38dd0 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -54,7 +54,6 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" - "github.com/pingcap/tidb/util/txnstateRecorder" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -583,9 +582,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { var lockKeyStats *util.LockKeysDetails ctx = context.WithValue(ctx, util.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() - txnstateRecorder.ReportBlocked(txn.StartTS()) err = txn.LockKeys(ctx, lockCtx, keys...) - txnstateRecorder.ReportUnblocked(txn.StartTS()) if lockKeyStats != nil { seVars.StmtCtx.MergeLockKeysExecDetails(lockKeyStats) } diff --git a/session/session.go b/session/session.go index 5741ffa01148d..301f5764850d5 100644 --- a/session/session.go +++ b/session/session.go @@ -507,9 +507,6 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(tikvstore.GuaranteeLinearizability, s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } - if s.txn.Valid() { - txnstateRecorder.ReportCommitStarted(s.txn.StartTS()) - } return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) } diff --git a/session/txn.go b/session/txn.go index 16f1309037849..e26efc4004028 100644 --- a/session/txn.go +++ b/session/txn.go @@ -235,6 +235,8 @@ func (txn *TxnState) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } + txnstateRecorder.ReportCommitStarted(txn.StartTS()) + // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { if val.(bool) { @@ -266,6 +268,13 @@ func (txn *TxnState) Rollback() error { return txn.Transaction.Rollback() } +func (txn *TxnState) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { + txnstateRecorder.ReportBlocked(txn.StartTS()) + err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) + txnstateRecorder.ReportUnblocked(txn.StartTS()) + return err +} + func (txn *TxnState) reset() { txn.cleanup() txn.changeToInvalid() From cf8a8897442fe5ac36ce90aaa317ba09b8492db0 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 22 Apr 2021 23:53:38 +0800 Subject: [PATCH 06/26] Collect txn information only when the user requests. Inline the txnStateEntry into txnstate Signed-off-by: longfangsong --- bindinfo/bind_test.go | 5 + domain/domain_test.go | 5 + executor/executor_pkg_test.go | 4 + executor/explainfor_test.go | 5 + executor/infoschema_reader.go | 10 +- executor/infoschema_reader_test.go | 5 + executor/prepared_test.go | 5 + executor/seqtest/prepared_test.go | 4 + infoschema/tables_test.go | 5 + server/server.go | 15 +- session/session.go | 22 +-- session/txn.go | 58 ++++++- util/processinfo.go | 2 + util/txnstateRecorder/txnstate_recorder.go | 187 --------------------- 14 files changed, 121 insertions(+), 211 deletions(-) delete mode 100644 util/txnstateRecorder/txnstate_recorder.go diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 1eb7e1478b2f9..a17a42414c5b1 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" utilparser "github.com/pingcap/tidb/util/parser" @@ -70,6 +71,10 @@ type mockSessionManager struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager) ShowTxnList() [][]types.Datum { + return nil +} + func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) for _, item := range msm.PS { diff --git a/domain/domain_test.go b/domain/domain_test.go index 7c9d9ff633bc5..ad34c399d57de 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" @@ -241,6 +242,10 @@ type mockSessionManager struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager) ShowTxnList() [][]types.Datum { + return nil +} + func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) for _, item := range msm.PS { diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index d3b7294b79845..fcb4d1b79dbd0 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -60,6 +60,10 @@ type mockSessionManager struct { serverID uint64 } +func (msm *mockSessionManager) ShowTxnList() [][]types.Datum { + return nil +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index a113200a925d8..e8a50d67e8f31 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/kvcache" @@ -38,6 +39,10 @@ type mockSessionManager1 struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager1) ShowTxnList() [][]types.Datum { + return nil +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 3288e434627bc..854269adabb74 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -52,13 +52,14 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" - "github.com/pingcap/tidb/util/txnstateRecorder" "go.etcd.io/etcd/clientv3" + "go.uber.org/zap" ) type memtableRetriever struct { @@ -1988,7 +1989,12 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context } func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) error { - e.rows = txnstateRecorder.Datums() + sm := ctx.GetSessionManager() + if sm == nil { + return nil + } + logutil.BgLogger().Info("----------", zap.Any("---", sm)) + e.rows = sm.ShowTxnList() return nil } diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index c3e125824873d..2029c4e8e77ff 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/stringutil" @@ -728,6 +729,10 @@ type mockSessionManager struct { serverID uint64 } +func (sm *mockSessionManager) ShowTxnList() [][]types.Datum { + return nil +} + func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { return sm.processInfoMap } diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 34cee0306d948..5c25c64dd7a8a 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -27,6 +27,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/testkit" ) @@ -131,6 +132,10 @@ type mockSessionManager2 struct { killed bool } +func (sm *mockSessionManager2) ShowTxnList() [][]types.Datum { + return nil +} + func (sm *mockSessionManager2) ShowProcessList() map[uint64]*util.ProcessInfo { pl := make(map[uint64]*util.ProcessInfo) if pi, ok := sm.GetProcessInfo(0); ok { diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 916f218db1f9d..240f5154c12d3 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -796,6 +796,10 @@ type mockSessionManager1 struct { Se session.Session } +func (msm *mockSessionManager1) ShowTxnList() [][]types.Datum { + return nil +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index f30f25ba6abfa..c93b5952fb7b7 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/pdapi" @@ -435,6 +436,10 @@ type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo } +func (sm *mockSessionManager) ShowTxnList() [][]types.Datum { + return nil +} + func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { return sm.processInfoMap } diff --git a/server/server.go b/server/server.go index f7a6021a11221..ea4b9ce4f42d8 100644 --- a/server/server.go +++ b/server/server.go @@ -37,8 +37,6 @@ import ( "math/rand" "net" "net/http" - "unsafe" - // For pprof _ "net/http/pprof" "os" @@ -46,6 +44,7 @@ import ( "sync" "sync/atomic" "time" + "unsafe" "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" @@ -58,6 +57,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/fastrand" @@ -557,6 +557,17 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { return rs } +// ShowTxnList shows all txn info for displaying in `TIDB_TRX` +func (s *Server) ShowTxnList() [][]types.Datum { + s.rwlock.RLock() + defer s.rwlock.RUnlock() + rs := make([][]types.Datum, len(s.clients)) + for _, client := range s.clients { + rs = append(rs, client.ctx.Session.TxnInfo()) + } + return rs +} + // GetProcessInfo implements the SessionManager interface. func (s *Server) GetProcessInfo(id uint64) (*util.ProcessInfo, bool) { s.rwlock.RLock() diff --git a/session/session.go b/session/session.go index 301f5764850d5..bdee659c10f2b 100644 --- a/session/session.go +++ b/session/session.go @@ -83,7 +83,6 @@ import ( "github.com/pingcap/tidb/util/sli" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" - "github.com/pingcap/tidb/util/txnstateRecorder" ) var ( @@ -146,6 +145,7 @@ type Session interface { Auth(user *auth.UserIdentity, auth []byte, salt []byte) bool AuthWithoutVerification(user *auth.UserIdentity) bool ShowProcess() *util.ProcessInfo + TxnInfo() []types.Datum // PrepareTxnCtx is exported for test. PrepareTxnCtx(context.Context) // FieldList returns fields list of a table. @@ -440,6 +440,10 @@ func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { return fields, nil } +func (s *session) TxnInfo() []types.Datum { + return s.txn.Datum() +} + func (s *session) doCommit(ctx context.Context) error { if !s.txn.Valid() { return nil @@ -600,10 +604,6 @@ func (s *session) CommitTxn(ctx context.Context) error { var commitDetail *tikvutil.CommitDetails ctx = context.WithValue(ctx, tikvutil.CommitDetailCtxKey, &commitDetail) - startTs := uint64(0) - if s.txn.Valid() { - startTs = s.txn.StartTS() - } err := s.doCommitWithRetry(ctx) if commitDetail != nil { s.sessionVars.StmtCtx.MergeExecDetails(nil, commitDetail) @@ -615,9 +615,6 @@ func (s *session) CommitTxn(ctx context.Context) error { } }) s.sessionVars.TxnCtx.Cleanup() - if startTs != 0 { - txnstateRecorder.ReportTxnEnd(startTs) - } return err } @@ -627,10 +624,8 @@ func (s *session) RollbackTxn(ctx context.Context) { defer span1.Finish() } - startTs := uint64(0) if s.txn.Valid() { - startTs = s.txn.StartTS() - txnstateRecorder.ReportRollbackStarted(startTs) + s.txn.State = TxnRollingBack terror.Log(s.txn.Rollback()) } if ctx.Value(inCloseSession{}) == nil { @@ -639,9 +634,6 @@ func (s *session) RollbackTxn(ctx context.Context) { s.txn.changeToInvalid() s.sessionVars.TxnCtx.Cleanup() s.sessionVars.SetInTxn(false) - if startTs != 0 { - txnstateRecorder.ReportTxnEnd(startTs) - } } func (s *session) GetClient() kv.Client { @@ -1406,7 +1398,7 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { if s.txn.Valid() { _, digest := parser.NormalizeDigest(stmtNode.Text()) - txnstateRecorder.ReportStatementStartExecute(s.txn.StartTS(), digest) + s.txn.CurrentSQLDigest = digest } if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { diff --git a/session/txn.go b/session/txn.go index e26efc4004028..0a6e76fe49692 100644 --- a/session/txn.go +++ b/session/txn.go @@ -20,11 +20,13 @@ import ( "runtime/trace" "strings" "sync/atomic" + "time" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" @@ -33,13 +35,27 @@ import ( tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sli" - "github.com/pingcap/tidb/util/txnstateRecorder" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" ) +// TxnRunningState is the current state of a transaction +type TxnRunningState = int + +const ( + // TxnRunningNormal means the transaction is running normally + TxnRunningNormal TxnRunningState = iota + // TxnLockWaiting means the transaction is blocked on a lock + TxnLockWaiting TxnRunningState = iota + // TxnCommitting means the transaction is (at least trying to) committing + TxnCommitting TxnRunningState = iota + // TxnRollingBack means the transaction is rolling back + TxnRollingBack TxnRunningState = iota +) + // TxnState wraps kv.Transaction to provide a new kv.Transaction. // 1. It holds all statement related modification in the buffer before flush to the txn, // so if execute statement meets error, the txn won't be made dirty. @@ -56,6 +72,19 @@ type TxnState struct { stagingHandle kv.StagingHandle mutations map[int64]*binlog.TableMutation writeSLI sli.TxnWriteThroughputSLI + // human readable startTime + startTime time.Time + // todo: Shall we parse startTs to get it? + // Pros: Save memory, some how is the "global" timestamp in a cluster(so for the CLUSTER_TIDB_TRX, this field would be more useful) + // Cons: May different with result of "NOW()" + humanReadableStartTime time.Time + // digest of SQL current running + CurrentSQLDigest string + // current executing state + State TxnRunningState + // last trying to block start time + // todo: currently even if stmtState is not Blocking, blockStartTime is not nil (showing last block), is it the preferred behaviour? + blockStartTime *time.Time } // GetTableInfo returns the cached index name. @@ -159,6 +188,7 @@ func (txn *TxnState) GoString() string { } func (txn *TxnState) changeInvalidToValid(kvTxn kv.Transaction) { + txn.startTime = time.Now() txn.Transaction = kvTxn txn.initStmtBuf() txn.txnFuture = nil @@ -183,8 +213,8 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { txn.Transaction = nil return err } + txn.startTime = time.Now() txn.Transaction = t - txnstateRecorder.ReportTxnStart(txn.StartTS()) txn.initStmtBuf() return nil } @@ -235,7 +265,7 @@ func (txn *TxnState) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } - txnstateRecorder.ReportCommitStarted(txn.StartTS()) + txn.State = TxnCommitting // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { @@ -268,10 +298,12 @@ func (txn *TxnState) Rollback() error { return txn.Transaction.Rollback() } +// LockKeys tries to lock the keys. Will block until all keys are locked successfully or an error occurs. func (txn *TxnState) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { - txnstateRecorder.ReportBlocked(txn.StartTS()) + originState := txn.State + txn.State = TxnLockWaiting err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) - txnstateRecorder.ReportUnblocked(txn.StartTS()) + txn.State = originState return err } @@ -327,6 +359,22 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { return !isNonUniqueIndex } +// Datum dump the TxnState to Datum for displaying in `TIDB_TRX` +func (txn *TxnState) Datum() []types.Datum { + var blockStartTime interface{} + if txn.blockStartTime == nil { + blockStartTime = nil + } else { + blockStartTime = types.NewTime(types.FromGoTime(*txn.blockStartTime), mysql.TypeTimestamp, 0) + } + return types.MakeDatums( + txn.StartTS(), + types.NewTime(types.FromGoTime(txn.humanReadableStartTime), mysql.TypeTimestamp, 0), + txn.CurrentSQLDigest, + txn.State, + blockStartTime) +} + func getBinlogMutation(ctx sessionctx.Context, tableID int64) *binlog.TableMutation { bin := binloginfo.GetPrewriteValue(ctx, true) for i := range bin.Mutations { diff --git a/util/processinfo.go b/util/processinfo.go index 29716d914c3de..6c3df62f412cb 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" ) @@ -161,6 +162,7 @@ func serverStatus2Str(state uint16) string { // kill statement rely on this interface. type SessionManager interface { ShowProcessList() map[uint64]*ProcessInfo + ShowTxnList() [][]types.Datum GetProcessInfo(id uint64) (*ProcessInfo, bool) Kill(connectionID uint64, query bool) KillAllConnections() diff --git a/util/txnstateRecorder/txnstate_recorder.go b/util/txnstateRecorder/txnstate_recorder.go deleted file mode 100644 index 8b60b051fe01f..0000000000000 --- a/util/txnstateRecorder/txnstate_recorder.go +++ /dev/null @@ -1,187 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package txnstateRecorder is for recording the transaction running state on current tidb instance -// so we can display them in `information_schema.TIDB_TRX` -package txnstateRecorder - -import ( - "sync" - "time" - - "github.com/pingcap/log" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/types" - "go.uber.org/zap" -) - -// TxnRunningState is the current state of a transaction -type TxnRunningState = int - -const ( - // TxnRunningNormal means the transaction is running normally - TxnRunningNormal TxnRunningState = iota - // TxnLockWaiting means the transaction is blocked on a lock - TxnLockWaiting TxnRunningState = iota - // TxnCommitting means the transaction is (at least trying to) committing - TxnCommitting TxnRunningState = iota - // TxnRollingBack means the transaction is rolling back - TxnRollingBack TxnRunningState = iota -) - -// txnStateEntry is an entry to be stored in `information_schema.TIDB_TRX` -type txnStateEntry struct { - // start timestamp of the transaction - // also worked as transaction id - startTs uint64 - // todo: Shall we parse startTs to get it? - // Pros: Save memory, some how is the "global" timestamp in a cluster(so for the CLUSTER_TIDB_TRX, this field would be more useful) - // Cons: May different with result of "NOW()" - humanReadableStartTime time.Time - // digest of SQL current running - currentSQLDigest string - // current executing state - state TxnRunningState - // last trying to block start time - // todo: currently even if stmtState is not Blocking, blockStartTime is not nil (showing last block), is it the preferred behaviour? - blockStartTime *time.Time -} - -// storage place for `information_schema.TIDB_TRX` -// todo: is it necessary to port executor/concurrent_map.go here? -type stateMap struct { - sync.Mutex - items map[uint64]*txnStateEntry -} - -var stateStorage = stateMap{ - items: map[uint64]*txnStateEntry{}, -} - -func (e *txnStateEntry) onStatementStartExecute(sqlDigest string) { - e.currentSQLDigest = sqlDigest -} - -func (e *txnStateEntry) onRollbackStarted() { - e.state = TxnRollingBack -} - -func (e *txnStateEntry) onCommitStarted() { - e.state = TxnCommitting -} - -func (e *txnStateEntry) onBlocked() { - e.state = TxnLockWaiting - now := time.Now() - e.blockStartTime = &now -} - -func (e *txnStateEntry) onUnblocked() { - e.state = TxnRunningNormal -} - -func (e *txnStateEntry) toDatum() []types.Datum { - var blockStartTime interface{} - if e.blockStartTime == nil { - blockStartTime = nil - } else { - blockStartTime = types.NewTime(types.FromGoTime(*e.blockStartTime), mysql.TypeTimestamp, 0) - } - return types.MakeDatums( - e.startTs, - types.NewTime(types.FromGoTime(e.humanReadableStartTime), mysql.TypeTimestamp, 0), - e.currentSQLDigest, - e.state, - blockStartTime) -} - -// ReportTxnStart is expected to be called when a transaction starts -func ReportTxnStart(txnID uint64) { - stateStorage.Lock() - defer stateStorage.Unlock() - stateStorage.items[txnID] = &txnStateEntry{ - startTs: txnID, - humanReadableStartTime: time.Now(), - currentSQLDigest: "", - state: TxnRunningNormal, - blockStartTime: nil, - } -} - -// ReportStatementStartExecute is expected to be called when a statement starts to run in a transaction -func ReportStatementStartExecute(txnID uint64, sqlDigest string) { - stateStorage.Lock() - defer stateStorage.Unlock() - if item, ok := stateStorage.items[txnID]; ok { - item.onStatementStartExecute(sqlDigest) - } -} - -// ReportRollbackStarted is expected to be called when a transaction starts to rollback -func ReportRollbackStarted(txnID uint64) { - stateStorage.Lock() - defer stateStorage.Unlock() - if item, ok := stateStorage.items[txnID]; ok { - item.onRollbackStarted() - } -} - -// ReportCommitStarted is expected to be called when a transaction starts to commit (call once for each try) -func ReportCommitStarted(txnID uint64) { - stateStorage.Lock() - defer stateStorage.Unlock() - if item, ok := stateStorage.items[txnID]; ok { - item.onCommitStarted() - } -} - -// ReportBlocked is expected to be called when a transaction is blocked when trying to acquiring a pessimistic lock -func ReportBlocked(txnID uint64) { - stateStorage.Lock() - defer stateStorage.Unlock() - if item, ok := stateStorage.items[txnID]; ok { - item.onBlocked() - } -} - -// ReportUnblocked is expected to be called when a transaction is unblocked -func ReportUnblocked(txnID uint64) { - stateStorage.Lock() - defer stateStorage.Unlock() - if item, ok := stateStorage.items[txnID]; ok { - item.onUnblocked() - } -} - -// ReportTxnEnd is expected to be called when a transaction end -func ReportTxnEnd(txnID uint64) { - stateStorage.Lock() - defer stateStorage.Unlock() - if _, ok := stateStorage.items[txnID]; ok { - delete(stateStorage.items, txnID) - } -} - -// Datums is used to read all the txn states to Datums, for querying by SQL -func Datums() [][]types.Datum { - stateStorage.Lock() - defer stateStorage.Unlock() - var result [][]types.Datum - for _, status := range stateStorage.items { - log.Info("stateStorage", zap.Any("item", status)) - if status != nil { - result = append(result, status.toDatum()) - } - } - return result -} From 5cd3f9467edadf960f8e9c1880b0e6a55c5c46f7 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Fri, 23 Apr 2021 10:16:57 +0800 Subject: [PATCH 07/26] Cleanup --- executor/infoschema_reader.go | 3 --- server/server.go | 10 ++++++++-- session/txn.go | 10 ++++++---- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 854269adabb74..19a55de4618c4 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -52,14 +52,12 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" "go.etcd.io/etcd/clientv3" - "go.uber.org/zap" ) type memtableRetriever struct { @@ -1993,7 +1991,6 @@ func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) error { if sm == nil { return nil } - logutil.BgLogger().Info("----------", zap.Any("---", sm)) e.rows = sm.ShowTxnList() return nil } diff --git a/server/server.go b/server/server.go index ea4b9ce4f42d8..846273a7af6f0 100644 --- a/server/server.go +++ b/server/server.go @@ -37,6 +37,7 @@ import ( "math/rand" "net" "net/http" + // For pprof _ "net/http/pprof" "os" @@ -561,9 +562,14 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { func (s *Server) ShowTxnList() [][]types.Datum { s.rwlock.RLock() defer s.rwlock.RUnlock() - rs := make([][]types.Datum, len(s.clients)) + rs := [][]types.Datum{} for _, client := range s.clients { - rs = append(rs, client.ctx.Session.TxnInfo()) + if client.ctx.Session != nil { + info := client.ctx.Session.TxnInfo() + if info != nil { + rs = append(rs, info) + } + } } return rs } diff --git a/session/txn.go b/session/txn.go index 0a6e76fe49692..4c14aa417169e 100644 --- a/session/txn.go +++ b/session/txn.go @@ -72,8 +72,6 @@ type TxnState struct { stagingHandle kv.StagingHandle mutations map[int64]*binlog.TableMutation writeSLI sli.TxnWriteThroughputSLI - // human readable startTime - startTime time.Time // todo: Shall we parse startTs to get it? // Pros: Save memory, some how is the "global" timestamp in a cluster(so for the CLUSTER_TIDB_TRX, this field would be more useful) // Cons: May different with result of "NOW()" @@ -188,7 +186,7 @@ func (txn *TxnState) GoString() string { } func (txn *TxnState) changeInvalidToValid(kvTxn kv.Transaction) { - txn.startTime = time.Now() + txn.humanReadableStartTime = time.Now() txn.Transaction = kvTxn txn.initStmtBuf() txn.txnFuture = nil @@ -213,7 +211,7 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { txn.Transaction = nil return err } - txn.startTime = time.Now() + txn.humanReadableStartTime = time.Now() txn.Transaction = t txn.initStmtBuf() return nil @@ -361,6 +359,10 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { // Datum dump the TxnState to Datum for displaying in `TIDB_TRX` func (txn *TxnState) Datum() []types.Datum { + logutil.BgLogger().Info("txnState.Datum", zap.String("txn", txn.GoString())) + if txn.pending() { + return nil + } var blockStartTime interface{} if txn.blockStartTime == nil { blockStartTime = nil From 51e0c4b385fee050d9b971ccacb2d4528c70bd73 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Sun, 25 Apr 2021 17:26:29 +0800 Subject: [PATCH 08/26] Several bug fixes --- session/session.go | 1 - session/txn.go | 18 ++--- .../txnstate_recorder_test.go | 69 ------------------- 3 files changed, 9 insertions(+), 79 deletions(-) delete mode 100644 util/txnstateRecorder/txnstate_recorder_test.go diff --git a/session/session.go b/session/session.go index bdee659c10f2b..28aab28dcfe41 100644 --- a/session/session.go +++ b/session/session.go @@ -625,7 +625,6 @@ func (s *session) RollbackTxn(ctx context.Context) { } if s.txn.Valid() { - s.txn.State = TxnRollingBack terror.Log(s.txn.Rollback()) } if ctx.Value(inCloseSession{}) == nil { diff --git a/session/txn.go b/session/txn.go index 4c14aa417169e..c37bb4ca3318b 100644 --- a/session/txn.go +++ b/session/txn.go @@ -72,16 +72,11 @@ type TxnState struct { stagingHandle kv.StagingHandle mutations map[int64]*binlog.TableMutation writeSLI sli.TxnWriteThroughputSLI - // todo: Shall we parse startTs to get it? - // Pros: Save memory, some how is the "global" timestamp in a cluster(so for the CLUSTER_TIDB_TRX, this field would be more useful) - // Cons: May different with result of "NOW()" - humanReadableStartTime time.Time // digest of SQL current running CurrentSQLDigest string // current executing state State TxnRunningState // last trying to block start time - // todo: currently even if stmtState is not Blocking, blockStartTime is not nil (showing last block), is it the preferred behaviour? blockStartTime *time.Time } @@ -97,6 +92,7 @@ func (txn *TxnState) CacheTableInfo(id int64, info *model.TableInfo) { func (txn *TxnState) init() { txn.mutations = make(map[int64]*binlog.TableMutation) + txn.State = TxnRunningNormal } func (txn *TxnState) initStmtBuf() { @@ -186,8 +182,8 @@ func (txn *TxnState) GoString() string { } func (txn *TxnState) changeInvalidToValid(kvTxn kv.Transaction) { - txn.humanReadableStartTime = time.Now() txn.Transaction = kvTxn + txn.State = TxnRunningNormal txn.initStmtBuf() txn.txnFuture = nil } @@ -211,8 +207,8 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { txn.Transaction = nil return err } - txn.humanReadableStartTime = time.Now() txn.Transaction = t + txn.State = TxnRunningNormal txn.initStmtBuf() return nil } @@ -293,6 +289,7 @@ func (txn *TxnState) Commit(ctx context.Context) error { // Rollback overrides the Transaction interface. func (txn *TxnState) Rollback() error { defer txn.reset() + txn.State = TxnRollingBack return txn.Transaction.Rollback() } @@ -300,7 +297,10 @@ func (txn *TxnState) Rollback() error { func (txn *TxnState) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { originState := txn.State txn.State = TxnLockWaiting + t := time.Now() + txn.blockStartTime = &t err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) + txn.blockStartTime = nil txn.State = originState return err } @@ -359,7 +359,6 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { // Datum dump the TxnState to Datum for displaying in `TIDB_TRX` func (txn *TxnState) Datum() []types.Datum { - logutil.BgLogger().Info("txnState.Datum", zap.String("txn", txn.GoString())) if txn.pending() { return nil } @@ -369,9 +368,10 @@ func (txn *TxnState) Datum() []types.Datum { } else { blockStartTime = types.NewTime(types.FromGoTime(*txn.blockStartTime), mysql.TypeTimestamp, 0) } + humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(txn.StartTS())*1e6) return types.MakeDatums( txn.StartTS(), - types.NewTime(types.FromGoTime(txn.humanReadableStartTime), mysql.TypeTimestamp, 0), + types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), txn.CurrentSQLDigest, txn.State, blockStartTime) diff --git a/util/txnstateRecorder/txnstate_recorder_test.go b/util/txnstateRecorder/txnstate_recorder_test.go deleted file mode 100644 index 9e7fd678ae714..0000000000000 --- a/util/txnstateRecorder/txnstate_recorder_test.go +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package txnstateRecorder is for recording the transaction running state on current tidb instance -// so we can display them in `information_schema.TIDB_TRX` -package txnstateRecorder_test - -import ( - "testing" - - . "github.com/pingcap/check" - - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" -) - -type testTxnStateRecorderSuite struct { - store kv.Storage -} - -var _ = Suite(&testTxnStateRecorderSuite{}) - -func TestT(t *testing.T) { - TestingT(t) -} - -func (s *testTxnStateRecorderSuite) SetUpSuite(c *C) { - var err error - s.store, err = mockstore.NewMockStore() - if err != nil { - panic(err) - } - _, err = session.BootstrapSession(s.store) - if err != nil { - panic(err) - } -} - -func (s *testTxnStateRecorderSuite) TearDownSuite(c *C) { - _ = s.store.Close() -} - -func (s *testTxnStateRecorderSuite) TestBasicRecordingTxnState(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test;") - tk.MustExec("create table t(id int, a varchar(128));") - tk.MustExec("insert into t(id, a) values (1, 'abcd');") - rows := tk.MustQuery("select * from information_schema.TIDB_TRX;").Rows() - c.Assert(len(rows), Equals, 0) - tk.MustExec("BEGIN;") - tk.MustExec("select * from t for update;") - rows = tk.MustQuery("select * from information_schema.TIDB_TRX;").Rows() - c.Assert(len(rows), Equals, 1) - tk.MustExec("COMMIT;") - rows = tk.MustQuery("select * from information_schema.TIDB_TRX;").Rows() - c.Assert(len(rows), Equals, 0) -} From 7926f5e16215110804beb984d5a21d35188cb561 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Mon, 26 Apr 2021 11:50:17 +0800 Subject: [PATCH 09/26] Basic Unit Test --- session/session_test.go | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/session/session_test.go b/session/session_test.go index 9ce875fa07868..f435ba58813f3 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -83,6 +83,7 @@ var _ = SerialSuites(&testSessionSerialSuite{}) var _ = SerialSuites(&testBackupRestoreSuite{}) var _ = Suite(&testClusteredSuite{}) var _ = SerialSuites(&testClusteredSerialSuite{}) +var _ = SerialSuites(&testTxnStateSuite{}) type testSessionSuiteBase struct { cluster cluster.Cluster @@ -4243,3 +4244,28 @@ func (s *testSessionSerialSuite) TestParseWithParams(c *C) { c.Assert(err, IsNil) c.Assert(sb.String(), Equals, "SELECT 3") } + +type testTxnStateSuite struct { + testSessionSuiteBase +} + +func (s *testTxnStateSuite) TestBasic(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1);") + tk.MustExec("begin;") + tk.MustExec("select * from t for update;") + info := tk.Se.TxnInfo() + // startTs + c.Assert(info[0].Kind(), Equals, types.KindUint64) + // human readable start time + c.Assert(info[1].Kind(), Equals, types.KindMysqlTime) + digest := info[2].GetString() + _, expectedDigest := parser.NormalizeDigest("select * from t for update;") + c.Assert(digest, Equals, expectedDigest) + // running state + c.Assert(info[3].GetInt64(), Equals, int64(session.TxnRunningNormal)) + // blockStartTime + c.Assert(info[4].IsNull(), Equals, true) +} From c1c9d6412e357f2d120869f7eae30c4006e12868 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Mon, 26 Apr 2021 15:53:35 +0800 Subject: [PATCH 10/26] test blocked trx --- session/session_test.go | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/session/session_test.go b/session/session_test.go index f435ba58813f3..e58509b60bac2 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4251,10 +4251,9 @@ type testTxnStateSuite struct { func (s *testTxnStateSuite) TestBasic(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test;") tk.MustExec("create table t(a int);") tk.MustExec("insert into t(a) values (1);") - tk.MustExec("begin;") + tk.MustExec("begin pessimistic;") tk.MustExec("select * from t for update;") info := tk.Se.TxnInfo() // startTs @@ -4269,3 +4268,18 @@ func (s *testTxnStateSuite) TestBasic(c *C) { // blockStartTime c.Assert(info[4].IsNull(), Equals, true) } + +func (s *testTxnStateSuite) TestBlocked(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1);") + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t where a = 1 for update;") + go func() { + tk2.MustExec("begin pessimistic") + tk2.MustExec("select * from t where a = 1 for update;") + }() + time.Sleep(200 * time.Millisecond) + c.Assert(tk2.Se.TxnInfo()[3].GetInt64(), Equals, int64(session.TxnLockWaiting)) +} From 9791e00ba88414fa4f770c7d6852c7936d8fe129 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Mon, 26 Apr 2021 17:24:46 +0800 Subject: [PATCH 11/26] Add columns from session --- infoschema/tables.go | 5 ++++- session/session.go | 16 +++++++++++++++- session/session_test.go | 12 ++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 554a72afc303a..243415e0c28cc 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1340,9 +1340,12 @@ var tableClientErrorsSummaryByHostCols = []columnInfo{ var tableTiDBTrxCols = []columnInfo{ {name: "ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, - {name: "SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, + {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeLonglong, size: 64, comment: "Current running state of the transaction"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, + {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, + {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, + {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, } // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. diff --git a/session/session.go b/session/session.go index b4013ca8523f5..04261b884e808 100644 --- a/session/session.go +++ b/session/session.go @@ -444,7 +444,21 @@ func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { } func (s *session) TxnInfo() []types.Datum { - return s.txn.Datum() + s.mu.RLock() + defer s.mu.RUnlock() + txnDatum := s.txn.Datum() + if txnDatum == nil { + return nil + } + var username interface{} = nil + if s.sessionVars.User != nil { + username = s.sessionVars.User.Username + } + return append(txnDatum, + types.NewDatum(s.sessionVars.ConnectionID), + types.NewDatum(username), + types.NewDatum(s.sessionVars.CurrentDB), + ) } func (s *session) doCommit(ctx context.Context) error { diff --git a/session/session_test.go b/session/session_test.go index e58509b60bac2..c3f0d0c20b3f2 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4267,6 +4267,13 @@ func (s *testTxnStateSuite) TestBasic(c *C) { c.Assert(info[3].GetInt64(), Equals, int64(session.TxnRunningNormal)) // blockStartTime c.Assert(info[4].IsNull(), Equals, true) + // sessionId + c.Assert(info[5].GetUint64(), Equals, tk.Se.GetSessionVars().ConnectionID) + // username + c.Assert(info[6].IsNull(), Equals, true) + // schema name + c.Assert(info[7].GetString(), Equals, "test") + tk.MustExec("COMMIT;") } func (s *testTxnStateSuite) TestBlocked(c *C) { @@ -4279,7 +4286,12 @@ func (s *testTxnStateSuite) TestBlocked(c *C) { go func() { tk2.MustExec("begin pessimistic") tk2.MustExec("select * from t where a = 1 for update;") + tk2.MustExec("COMMIT;") }() time.Sleep(200 * time.Millisecond) + // state c.Assert(tk2.Se.TxnInfo()[3].GetInt64(), Equals, int64(session.TxnLockWaiting)) + // blockStartTime + c.Assert(tk2.Se.TxnInfo()[4].IsNull(), Equals, false) + tk.MustExec("COMMIT;") } From 78e8a4cb7520b500670e218e2b55e3a7d9dbc4fb Mon Sep 17 00:00:00 2001 From: longfangsong Date: Mon, 26 Apr 2021 18:02:01 +0800 Subject: [PATCH 12/26] Save transaction length and size --- infoschema/tables.go | 2 ++ session/session_test.go | 28 ++++++++++++++++++++++------ session/txn.go | 5 ++++- 3 files changed, 28 insertions(+), 7 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 243415e0c28cc..2b574e46147ba 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1343,6 +1343,8 @@ var tableTiDBTrxCols = []columnInfo{ {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeLonglong, size: 64, comment: "Current running state of the transaction"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, + {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "entries in MemDB"}, + {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used size"}, {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, diff --git a/session/session_test.go b/session/session_test.go index c3f0d0c20b3f2..20ea70c469bcc 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4267,13 +4267,14 @@ func (s *testTxnStateSuite) TestBasic(c *C) { c.Assert(info[3].GetInt64(), Equals, int64(session.TxnRunningNormal)) // blockStartTime c.Assert(info[4].IsNull(), Equals, true) + // [5] and [6] are len and size, which will covered in TestLenAndSize // sessionId - c.Assert(info[5].GetUint64(), Equals, tk.Se.GetSessionVars().ConnectionID) + c.Assert(info[7].GetUint64(), Equals, tk.Se.GetSessionVars().ConnectionID) // username - c.Assert(info[6].IsNull(), Equals, true) + c.Assert(info[8].IsNull(), Equals, true) // schema name - c.Assert(info[7].GetString(), Equals, "test") - tk.MustExec("COMMIT;") + c.Assert(info[9].GetString(), Equals, "test") + tk.MustExec("commit;") } func (s *testTxnStateSuite) TestBlocked(c *C) { @@ -4286,12 +4287,27 @@ func (s *testTxnStateSuite) TestBlocked(c *C) { go func() { tk2.MustExec("begin pessimistic") tk2.MustExec("select * from t where a = 1 for update;") - tk2.MustExec("COMMIT;") + tk2.MustExec("commit;") }() time.Sleep(200 * time.Millisecond) // state c.Assert(tk2.Se.TxnInfo()[3].GetInt64(), Equals, int64(session.TxnLockWaiting)) // blockStartTime c.Assert(tk2.Se.TxnInfo()[4].IsNull(), Equals, false) - tk.MustExec("COMMIT;") + tk.MustExec("commit;") +} + +func (s *testTxnStateSuite) TestLenAndSize(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("begin pessimistic;") + tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info[5].GetInt64(), Equals, int64(1)) + c.Assert(info[6].GetInt64(), Equals, int64(29)) + tk.MustExec("insert into t(a) values (2);") + info = tk.Se.TxnInfo() + c.Assert(info[5].GetInt64(), Equals, int64(2)) + c.Assert(info[6].GetInt64(), Equals, int64(58)) + tk.MustExec("commit;") } diff --git a/session/txn.go b/session/txn.go index c37bb4ca3318b..f148525359965 100644 --- a/session/txn.go +++ b/session/txn.go @@ -374,7 +374,10 @@ func (txn *TxnState) Datum() []types.Datum { types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), txn.CurrentSQLDigest, txn.State, - blockStartTime) + blockStartTime, + txn.Transaction.Len(), + txn.Transaction.Size(), + ) } func getBinlogMutation(ctx sessionctx.Context, tableID int64) *binlog.TableMutation { From 868f6f1c7258c14cd1100ca99755b33089b0b847 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Wed, 28 Apr 2021 10:55:45 +0800 Subject: [PATCH 13/26] Cleanup and address some comments --- executor/builder.go | 1 - kv/kv.go | 1 + session/txn.go | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index c5e8ae3b91b80..e6e4d2e4f9e9f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1545,7 +1545,6 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), strings.ToLower(infoschema.TableClientErrorsSummaryByUser), strings.ToLower(infoschema.TableClientErrorsSummaryByHost), - strings.ToLower(infoschema.TablePlacementPolicy), strings.ToLower(infoschema.TableTiDBTrx), strings.ToLower(infoschema.ClusterTableTiDBTrx): return &MemTableReaderExec{ diff --git a/kv/kv.go b/kv/kv.go index 711dd4e3ee860..01345ec357a97 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -154,6 +154,7 @@ type Transaction interface { // String implements fmt.Stringer interface. String() string // LockKeys tries to lock the entries with the keys in KV store. + // Will block until all keys are locked successfully or an error occurs. LockKeys(ctx context.Context, lockCtx *LockCtx, keys ...Key) error // SetOption sets an option with a value, when val is nil, uses the default // value of this option. diff --git a/session/txn.go b/session/txn.go index f148525359965..9a698bc22ab7f 100644 --- a/session/txn.go +++ b/session/txn.go @@ -293,7 +293,7 @@ func (txn *TxnState) Rollback() error { return txn.Transaction.Rollback() } -// LockKeys tries to lock the keys. Will block until all keys are locked successfully or an error occurs. +// Wrap the inner transaction's `LockKeys` to record the status func (txn *TxnState) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { originState := txn.State txn.State = TxnLockWaiting From 0cdf3a07b7e7aa1cc261fb55c43ca29272498fd9 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Wed, 28 Apr 2021 13:12:27 +0800 Subject: [PATCH 14/26] extract TxnInfo struct Note TxnInfo has to be in a standalone mod to prevent cycle import --- bindinfo/bind_test.go | 6 +- domain/domain_test.go | 6 +- executor/executor_pkg_test.go | 5 +- executor/explainfor_test.go | 4 +- executor/infoschema_reader.go | 5 +- executor/infoschema_reader_test.go | 6 +- executor/prepared_test.go | 6 +- executor/seqtest/prepared_test.go | 5 +- infoschema/tables.go | 4 +- infoschema/tables_test.go | 6 +- server/server.go | 8 +-- session/session.go | 25 ++++--- session/session_test.go | 39 ++++------- session/txn.go | 109 ++++++++++++----------------- session/txnInfo/txnInfo.go | 81 +++++++++++++++++++++ util/processinfo.go | 4 +- 16 files changed, 185 insertions(+), 134 deletions(-) create mode 100644 session/txnInfo/txnInfo.go diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index a17a42414c5b1..443ae04cfcbe0 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -35,9 +35,9 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" utilparser "github.com/pingcap/tidb/util/parser" @@ -71,8 +71,8 @@ type mockSessionManager struct { PS []*util.ProcessInfo } -func (msm *mockSessionManager) ShowTxnList() [][]types.Datum { - return nil +func (msm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { diff --git a/domain/domain_test.go b/domain/domain_test.go index ad34c399d57de..afd4bf95fc532 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -35,11 +35,11 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" @@ -242,8 +242,8 @@ type mockSessionManager struct { PS []*util.ProcessInfo } -func (msm *mockSessionManager) ShowTxnList() [][]types.Datum { - return nil +func (msm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index fea7f388f70ff..39ea2f47e1833 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" plannerutil "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -60,8 +61,8 @@ type mockSessionManager struct { serverID uint64 } -func (msm *mockSessionManager) ShowTxnList() [][]types.Datum { - return nil +func (msm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } // ShowProcessList implements the SessionManager.ShowProcessList interface. diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index e8a50d67e8f31..5a9a3df422738 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/kvcache" @@ -39,7 +39,7 @@ type mockSessionManager1 struct { PS []*util.ProcessInfo } -func (msm *mockSessionManager1) ShowTxnList() [][]types.Datum { +func (msm *mockSessionManager1) ShowTxnList() []txnInfo.TxnInfo { return nil } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 7100ffd2292a7..b6c2ee46631a2 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1991,7 +1991,10 @@ func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) error { if sm == nil { return nil } - e.rows = sm.ShowTxnList() + infoList := sm.ShowTxnList() + for _, info := range infoList { + e.rows = append(e.rows, info.ToDatum()) + } return nil } diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 2029c4e8e77ff..8b8e0fa973fb9 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -36,12 +36,12 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/stringutil" @@ -729,8 +729,8 @@ type mockSessionManager struct { serverID uint64 } -func (sm *mockSessionManager) ShowTxnList() [][]types.Datum { - return nil +func (sm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { diff --git a/executor/prepared_test.go b/executor/prepared_test.go index f51d517876e09..ee0d8e6922c13 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/tidb/domain" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" @@ -136,8 +136,8 @@ type mockSessionManager2 struct { killed bool } -func (sm *mockSessionManager2) ShowTxnList() [][]types.Datum { - return nil +func (sm *mockSessionManager2) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } func (sm *mockSessionManager2) ShowProcessList() map[uint64]*util.ProcessInfo { diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 240f5154c12d3..7ab408eed754f 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" @@ -796,8 +797,8 @@ type mockSessionManager1 struct { Se session.Session } -func (msm *mockSessionManager1) ShowTxnList() [][]types.Datum { - return nil +func (msm *mockSessionManager1) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } // ShowProcessList implements the SessionManager.ShowProcessList interface. diff --git a/infoschema/tables.go b/infoschema/tables.go index 2b574e46147ba..2a81a2eb8d7f1 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1343,8 +1343,8 @@ var tableTiDBTrxCols = []columnInfo{ {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeLonglong, size: 64, comment: "Current running state of the transaction"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, - {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "entries in MemDB"}, - {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used size"}, + {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, + {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index c93b5952fb7b7..5dcf18338d86d 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -42,9 +42,9 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/pdapi" @@ -436,8 +436,8 @@ type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo } -func (sm *mockSessionManager) ShowTxnList() [][]types.Datum { - return nil +func (sm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { + panic("unimplemented!") } func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { diff --git a/server/server.go b/server/server.go index 846273a7af6f0..3e5b60f83865c 100644 --- a/server/server.go +++ b/server/server.go @@ -56,9 +56,9 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/plugin" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/fastrand" @@ -559,15 +559,15 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { } // ShowTxnList shows all txn info for displaying in `TIDB_TRX` -func (s *Server) ShowTxnList() [][]types.Datum { +func (s *Server) ShowTxnList() []txnInfo.TxnInfo { s.rwlock.RLock() defer s.rwlock.RUnlock() - rs := [][]types.Datum{} + rs := []txnInfo.TxnInfo{} for _, client := range s.clients { if client.ctx.Session != nil { info := client.ctx.Session.TxnInfo() if info != nil { - rs = append(rs, info) + rs = append(rs, *info) } } } diff --git a/session/session.go b/session/session.go index 1d7792a00f0f1..dec429de43e1e 100644 --- a/session/session.go +++ b/session/session.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -145,7 +146,8 @@ type Session interface { Auth(user *auth.UserIdentity, auth []byte, salt []byte) bool AuthWithoutVerification(user *auth.UserIdentity) bool ShowProcess() *util.ProcessInfo - TxnInfo() []types.Datum + // Return the information of the txn current running + TxnInfo() *txnInfo.TxnInfo // PrepareTxnCtx is exported for test. PrepareTxnCtx(context.Context) // FieldList returns fields list of a table. @@ -184,7 +186,7 @@ func (h *StmtHistory) Count() int { type session struct { // processInfo is used by ShowProcess(), and should be modified atomically. processInfo atomic.Value - txn TxnState + txn LazyTxn mu struct { sync.RWMutex @@ -443,22 +445,19 @@ func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { return fields, nil } -func (s *session) TxnInfo() []types.Datum { - s.mu.RLock() - defer s.mu.RUnlock() - txnDatum := s.txn.Datum() - if txnDatum == nil { +func (s *session) TxnInfo() *txnInfo.TxnInfo { + txnInfo := s.txn.Info() + if txnInfo == nil { return nil } - var username interface{} = nil + var username string = "" if s.sessionVars.User != nil { username = s.sessionVars.User.Username } - return append(txnDatum, - types.NewDatum(s.sessionVars.ConnectionID), - types.NewDatum(username), - types.NewDatum(s.sessionVars.CurrentDB), - ) + txnInfo.ConnectionID = s.sessionVars.ConnectionID + txnInfo.Username = username + txnInfo.CurrentDB = s.sessionVars.CurrentDB + return txnInfo } func (s *session) doCommit(ctx context.Context) error { diff --git a/session/session_test.go b/session/session_test.go index 20ea70c469bcc..b88853186d11b 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -42,6 +42,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" @@ -4256,24 +4257,14 @@ func (s *testTxnStateSuite) TestBasic(c *C) { tk.MustExec("begin pessimistic;") tk.MustExec("select * from t for update;") info := tk.Se.TxnInfo() - // startTs - c.Assert(info[0].Kind(), Equals, types.KindUint64) - // human readable start time - c.Assert(info[1].Kind(), Equals, types.KindMysqlTime) - digest := info[2].GetString() _, expectedDigest := parser.NormalizeDigest("select * from t for update;") - c.Assert(digest, Equals, expectedDigest) - // running state - c.Assert(info[3].GetInt64(), Equals, int64(session.TxnRunningNormal)) - // blockStartTime - c.Assert(info[4].IsNull(), Equals, true) - // [5] and [6] are len and size, which will covered in TestLenAndSize - // sessionId - c.Assert(info[7].GetUint64(), Equals, tk.Se.GetSessionVars().ConnectionID) - // username - c.Assert(info[8].IsNull(), Equals, true) - // schema name - c.Assert(info[9].GetString(), Equals, "test") + c.Assert(info.CurrentSQLDigest, Equals, expectedDigest) + c.Assert(info.State, Equals, txnInfo.TxnRunningNormal) + c.Assert(info.BlockStartTime, IsNil) + // len and size will be covered in TestLenAndSize + c.Assert(info.ConnectionID, Equals, tk.Se.GetSessionVars().ConnectionID) + c.Assert(info.Username, Equals, "") + c.Assert(info.CurrentDB, Equals, "test") tk.MustExec("commit;") } @@ -4290,10 +4281,8 @@ func (s *testTxnStateSuite) TestBlocked(c *C) { tk2.MustExec("commit;") }() time.Sleep(200 * time.Millisecond) - // state - c.Assert(tk2.Se.TxnInfo()[3].GetInt64(), Equals, int64(session.TxnLockWaiting)) - // blockStartTime - c.Assert(tk2.Se.TxnInfo()[4].IsNull(), Equals, false) + c.Assert(tk2.Se.TxnInfo().State, Equals, txnInfo.TxnLockWaiting) + c.Assert(tk2.Se.TxnInfo().BlockStartTime, NotNil) tk.MustExec("commit;") } @@ -4303,11 +4292,11 @@ func (s *testTxnStateSuite) TestLenAndSize(c *C) { tk.MustExec("begin pessimistic;") tk.MustExec("insert into t(a) values (1);") info := tk.Se.TxnInfo() - c.Assert(info[5].GetInt64(), Equals, int64(1)) - c.Assert(info[6].GetInt64(), Equals, int64(29)) + c.Assert(info.Len, Equals, 1) + c.Assert(info.Size, Equals, 29) tk.MustExec("insert into t(a) values (2);") info = tk.Se.TxnInfo() - c.Assert(info[5].GetInt64(), Equals, int64(2)) - c.Assert(info[6].GetInt64(), Equals, int64(58)) + c.Assert(info.Len, Equals, 2) + c.Assert(info.Size, Equals, 58) tk.MustExec("commit;") } diff --git a/session/txn.go b/session/txn.go index 9a698bc22ab7f..4f6ba7582303f 100644 --- a/session/txn.go +++ b/session/txn.go @@ -26,42 +26,27 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sli" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" ) -// TxnRunningState is the current state of a transaction -type TxnRunningState = int - -const ( - // TxnRunningNormal means the transaction is running normally - TxnRunningNormal TxnRunningState = iota - // TxnLockWaiting means the transaction is blocked on a lock - TxnLockWaiting TxnRunningState = iota - // TxnCommitting means the transaction is (at least trying to) committing - TxnCommitting TxnRunningState = iota - // TxnRollingBack means the transaction is rolling back - TxnRollingBack TxnRunningState = iota -) - -// TxnState wraps kv.Transaction to provide a new kv.Transaction. +// LazyTxn wraps kv.Transaction to provide a new kv.Transaction. // 1. It holds all statement related modification in the buffer before flush to the txn, // so if execute statement meets error, the txn won't be made dirty. // 2. It's a lazy transaction, that means it's a txnFuture before StartTS() is really need. -type TxnState struct { - // States of a TxnState should be one of the followings: +type LazyTxn struct { + // States of a LazyTxn should be one of the followings: // Invalid: kv.Transaction == nil && txnFuture == nil // Pending: kv.Transaction == nil && txnFuture != nil // Valid: kv.Transaction != nil && txnFuture == nil @@ -75,27 +60,27 @@ type TxnState struct { // digest of SQL current running CurrentSQLDigest string // current executing state - State TxnRunningState + State txnInfo.TxnRunningState // last trying to block start time blockStartTime *time.Time } // GetTableInfo returns the cached index name. -func (txn *TxnState) GetTableInfo(id int64) *model.TableInfo { +func (txn *LazyTxn) GetTableInfo(id int64) *model.TableInfo { return txn.Transaction.GetTableInfo(id) } // CacheTableInfo caches the index name. -func (txn *TxnState) CacheTableInfo(id int64, info *model.TableInfo) { +func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { txn.Transaction.CacheTableInfo(id, info) } -func (txn *TxnState) init() { +func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) - txn.State = TxnRunningNormal + txn.State = txnInfo.TxnRunningNormal } -func (txn *TxnState) initStmtBuf() { +func (txn *LazyTxn) initStmtBuf() { if txn.Transaction == nil { return } @@ -105,14 +90,14 @@ func (txn *TxnState) initStmtBuf() { } // countHint is estimated count of mutations. -func (txn *TxnState) countHint() int { +func (txn *LazyTxn) countHint() int { if txn.stagingHandle == kv.InvalidStagingHandle { return 0 } return txn.Transaction.GetMemBuffer().Len() - txn.initCnt } -func (txn *TxnState) flushStmtBuf() { +func (txn *LazyTxn) flushStmtBuf() { if txn.stagingHandle == kv.InvalidStagingHandle { return } @@ -121,7 +106,7 @@ func (txn *TxnState) flushStmtBuf() { txn.initCnt = buf.Len() } -func (txn *TxnState) cleanupStmtBuf() { +func (txn *LazyTxn) cleanupStmtBuf() { if txn.stagingHandle == kv.InvalidStagingHandle { return } @@ -131,7 +116,7 @@ func (txn *TxnState) cleanupStmtBuf() { } // Size implements the MemBuffer interface. -func (txn *TxnState) Size() int { +func (txn *LazyTxn) Size() int { if txn.Transaction == nil { return 0 } @@ -139,19 +124,19 @@ func (txn *TxnState) Size() int { } // Valid implements the kv.Transaction interface. -func (txn *TxnState) Valid() bool { +func (txn *LazyTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() } -func (txn *TxnState) pending() bool { +func (txn *LazyTxn) pending() bool { return txn.Transaction == nil && txn.txnFuture != nil } -func (txn *TxnState) validOrPending() bool { +func (txn *LazyTxn) validOrPending() bool { return txn.txnFuture != nil || txn.Valid() } -func (txn *TxnState) String() string { +func (txn *LazyTxn) String() string { if txn.Transaction != nil { return txn.Transaction.String() } @@ -162,7 +147,7 @@ func (txn *TxnState) String() string { } // GoString implements the "%#v" format for fmt.Printf. -func (txn *TxnState) GoString() string { +func (txn *LazyTxn) GoString() string { var s strings.Builder s.WriteString("Txn{") if txn.pending() { @@ -181,19 +166,19 @@ func (txn *TxnState) GoString() string { return s.String() } -func (txn *TxnState) changeInvalidToValid(kvTxn kv.Transaction) { +func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { txn.Transaction = kvTxn - txn.State = TxnRunningNormal + txn.State = txnInfo.TxnRunningNormal txn.initStmtBuf() txn.txnFuture = nil } -func (txn *TxnState) changeInvalidToPending(future *txnFuture) { +func (txn *LazyTxn) changeInvalidToPending(future *txnFuture) { txn.Transaction = nil txn.txnFuture = future } -func (txn *TxnState) changePendingToValid(ctx context.Context) error { +func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { if txn.txnFuture == nil { return errors.New("transaction future is not set") } @@ -208,12 +193,12 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t - txn.State = TxnRunningNormal + txn.State = txnInfo.TxnRunningNormal txn.initStmtBuf() return nil } -func (txn *TxnState) changeToInvalid() { +func (txn *LazyTxn) changeToInvalid() { if txn.stagingHandle != kv.InvalidStagingHandle { txn.Transaction.GetMemBuffer().Cleanup(txn.stagingHandle) } @@ -249,7 +234,7 @@ func ResetMockAutoRandIDRetryCount(failTimes int64) { } // Commit overrides the Transaction interface. -func (txn *TxnState) Commit(ctx context.Context) error { +func (txn *LazyTxn) Commit(ctx context.Context) error { defer txn.reset() if len(txn.mutations) != 0 || txn.countHint() != 0 { logutil.BgLogger().Error("the code should never run here", @@ -259,7 +244,7 @@ func (txn *TxnState) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } - txn.State = TxnCommitting + txn.State = txnInfo.TxnCommitting // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { @@ -287,16 +272,16 @@ func (txn *TxnState) Commit(ctx context.Context) error { } // Rollback overrides the Transaction interface. -func (txn *TxnState) Rollback() error { +func (txn *LazyTxn) Rollback() error { defer txn.reset() - txn.State = TxnRollingBack + txn.State = txnInfo.TxnRollingBack return txn.Transaction.Rollback() } // Wrap the inner transaction's `LockKeys` to record the status -func (txn *TxnState) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { +func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { originState := txn.State - txn.State = TxnLockWaiting + txn.State = txnInfo.TxnLockWaiting t := time.Now() txn.blockStartTime = &t err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) @@ -305,12 +290,12 @@ func (txn *TxnState) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ... return err } -func (txn *TxnState) reset() { +func (txn *LazyTxn) reset() { txn.cleanup() txn.changeToInvalid() } -func (txn *TxnState) cleanup() { +func (txn *LazyTxn) cleanup() { txn.cleanupStmtBuf() txn.initStmtBuf() for key := range txn.mutations { @@ -319,7 +304,7 @@ func (txn *TxnState) cleanup() { } // KeysNeedToLock returns the keys need to be locked. -func (txn *TxnState) KeysNeedToLock() ([]kv.Key, error) { +func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) { if txn.stagingHandle == kv.InvalidStagingHandle { return nil, nil } @@ -358,26 +343,18 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { } // Datum dump the TxnState to Datum for displaying in `TIDB_TRX` -func (txn *TxnState) Datum() []types.Datum { - if txn.pending() { +func (txn *LazyTxn) Info() *txnInfo.TxnInfo { + if !txn.Valid() { return nil } - var blockStartTime interface{} - if txn.blockStartTime == nil { - blockStartTime = nil - } else { - blockStartTime = types.NewTime(types.FromGoTime(*txn.blockStartTime), mysql.TypeTimestamp, 0) + return &txnInfo.TxnInfo{ + StartTS: txn.StartTS(), + CurrentSQLDigest: txn.CurrentSQLDigest, + State: txn.State, + BlockStartTime: txn.blockStartTime, + Len: txn.Transaction.Len(), + Size: txn.Transaction.Size(), } - humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(txn.StartTS())*1e6) - return types.MakeDatums( - txn.StartTS(), - types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), - txn.CurrentSQLDigest, - txn.State, - blockStartTime, - txn.Transaction.Len(), - txn.Transaction.Size(), - ) } func getBinlogMutation(ctx sessionctx.Context, tableID int64) *binlog.TableMutation { diff --git a/session/txnInfo/txnInfo.go b/session/txnInfo/txnInfo.go new file mode 100644 index 0000000000000..ca98e6ed81f21 --- /dev/null +++ b/session/txnInfo/txnInfo.go @@ -0,0 +1,81 @@ +// Copyright 2021 PingCAP, Inc. + +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package txnInfo + +import ( + "time" + + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" +) + +// TxnRunningState is the current state of a transaction +type TxnRunningState = int + +const ( + // TxnRunningNormal means the transaction is running normally + TxnRunningNormal TxnRunningState = iota + // TxnLockWaiting means the transaction is blocked on a lock + TxnLockWaiting TxnRunningState = iota + // TxnCommitting means the transaction is (at least trying to) committing + TxnCommitting TxnRunningState = iota + // TxnRollingBack means the transaction is rolling back + TxnRollingBack TxnRunningState = iota +) + +// Infomation about a running transaction +// This is supposed to be the datasource of `TIDB_TRX` in infoschema +type TxnInfo struct { + StartTS uint64 + // digest of SQL current running + CurrentSQLDigest string + // current executing State + State TxnRunningState + // last trying to block start time + BlockStartTime *time.Time + // How many entries are in MemDB + Len int + // MemDB used memory + Size int + + // the following fields will be filled in `session` instead of `LazyTxn` + + // Which session this transaction belongs to + ConnectionID uint64 + // The user who open this session + Username string + // The schema this transaction works on + CurrentDB string +} + +// Convert the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table +func (info TxnInfo) ToDatum() []types.Datum { + humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) + var blockStartTime interface{} + if info.BlockStartTime == nil { + blockStartTime = nil + } else { + blockStartTime = types.NewTime(types.FromGoTime(*info.BlockStartTime), mysql.TypeTimestamp, 0) + } + return types.MakeDatums( + info.StartTS, + types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), + info.CurrentSQLDigest, + info.State, + blockStartTime, + info.Len, + info.Size, + ) +} diff --git a/util/processinfo.go b/util/processinfo.go index 6c3df62f412cb..5daeb96e04639 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -22,9 +22,9 @@ import ( "time" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" ) @@ -162,7 +162,7 @@ func serverStatus2Str(state uint16) string { // kill statement rely on this interface. type SessionManager interface { ShowProcessList() map[uint64]*ProcessInfo - ShowTxnList() [][]types.Datum + ShowTxnList() []txnInfo.TxnInfo GetProcessInfo(id uint64) (*ProcessInfo, bool) Kill(connectionID uint64, query bool) KillAllConnections() From c6f41b4dd8919a17b7b2a07494363e88f6fd2bf6 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Wed, 28 Apr 2021 13:32:11 +0800 Subject: [PATCH 15/26] make lint happy --- session/txn.go | 4 ++-- session/txnInfo/txnInfo.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/session/txn.go b/session/txn.go index 4f6ba7582303f..8f14bb365c23a 100644 --- a/session/txn.go +++ b/session/txn.go @@ -278,7 +278,7 @@ func (txn *LazyTxn) Rollback() error { return txn.Transaction.Rollback() } -// Wrap the inner transaction's `LockKeys` to record the status +// LockKeys Wrap the inner transaction's `LockKeys` to record the status func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { originState := txn.State txn.State = txnInfo.TxnLockWaiting @@ -342,7 +342,7 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { return !isNonUniqueIndex } -// Datum dump the TxnState to Datum for displaying in `TIDB_TRX` +// Info dump the TxnState to Datum for displaying in `TIDB_TRX` func (txn *LazyTxn) Info() *txnInfo.TxnInfo { if !txn.Valid() { return nil diff --git a/session/txnInfo/txnInfo.go b/session/txnInfo/txnInfo.go index ca98e6ed81f21..d1f79d8254a82 100644 --- a/session/txnInfo/txnInfo.go +++ b/session/txnInfo/txnInfo.go @@ -35,7 +35,7 @@ const ( TxnRollingBack TxnRunningState = iota ) -// Infomation about a running transaction +// TxnInfo is information about a running transaction // This is supposed to be the datasource of `TIDB_TRX` in infoschema type TxnInfo struct { StartTS uint64 @@ -60,7 +60,7 @@ type TxnInfo struct { CurrentDB string } -// Convert the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table +// ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table func (info TxnInfo) ToDatum() []types.Datum { humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) var blockStartTime interface{} From 9a8834fde78c47747e3e3b0fa5b27f32ac14f011 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Fri, 30 Apr 2021 11:12:56 +0800 Subject: [PATCH 16/26] Make LazyTxn.Info thread safe --- session/session.go | 13 +++----- session/session_test.go | 14 +++++--- session/txn.go | 48 ++++++++++++++++++---------- session/txnInfo/txnInfo.go | 6 ++-- store/tikv/unionstore/memdb.go | 19 +++++------ store/tikv/unionstore/memdb_arena.go | 5 +-- 6 files changed, 61 insertions(+), 44 deletions(-) diff --git a/session/session.go b/session/session.go index dec429de43e1e..f932edc0561c7 100644 --- a/session/session.go +++ b/session/session.go @@ -450,13 +450,10 @@ func (s *session) TxnInfo() *txnInfo.TxnInfo { if txnInfo == nil { return nil } - var username string = "" - if s.sessionVars.User != nil { - username = s.sessionVars.User.Username - } - txnInfo.ConnectionID = s.sessionVars.ConnectionID - txnInfo.Username = username - txnInfo.CurrentDB = s.sessionVars.CurrentDB + processInfo := s.ShowProcess() + txnInfo.ConnectionID = processInfo.ID + txnInfo.Username = processInfo.User + txnInfo.CurrentDB = processInfo.DB return txnInfo } @@ -1413,7 +1410,7 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { if s.txn.Valid() { _, digest := parser.NormalizeDigest(stmtNode.Text()) - s.txn.CurrentSQLDigest = digest + s.txn.CurrentSQLDigest.Store(&digest) } if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { diff --git a/session/session_test.go b/session/session_test.go index 55d4f61ae85de..d12e3ee61f93c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4254,9 +4254,11 @@ func (s *testTxnStateSuite) TestBasic(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info, IsNil) tk.MustExec("begin pessimistic;") tk.MustExec("select * from t for update;") - info := tk.Se.TxnInfo() + info = tk.Se.TxnInfo() _, expectedDigest := parser.NormalizeDigest("select * from t for update;") c.Assert(info.CurrentSQLDigest, Equals, expectedDigest) c.Assert(info.State, Equals, txnInfo.TxnRunningNormal) @@ -4266,6 +4268,8 @@ func (s *testTxnStateSuite) TestBasic(c *C) { c.Assert(info.Username, Equals, "") c.Assert(info.CurrentDB, Equals, "test") tk.MustExec("commit;") + info = tk.Se.TxnInfo() + c.Assert(info, IsNil) } func (s *testTxnStateSuite) TestBlocked(c *C) { @@ -4292,11 +4296,11 @@ func (s *testTxnStateSuite) TestLenAndSize(c *C) { tk.MustExec("begin pessimistic;") tk.MustExec("insert into t(a) values (1);") info := tk.Se.TxnInfo() - c.Assert(info.Len, Equals, 1) - c.Assert(info.Size, Equals, 29) + c.Assert(info.Len, Equals, int64(1)) + c.Assert(info.Size, Equals, int64(29)) tk.MustExec("insert into t(a) values (2);") info = tk.Se.TxnInfo() - c.Assert(info.Len, Equals, 2) - c.Assert(info.Size, Equals, 58) + c.Assert(info.Len, Equals, int64(2)) + c.Assert(info.Size, Equals, int64(58)) tk.MustExec("commit;") } diff --git a/session/txn.go b/session/txn.go index 8f14bb365c23a..ba2d73d47aa4a 100644 --- a/session/txn.go +++ b/session/txn.go @@ -21,6 +21,7 @@ import ( "strings" "sync/atomic" "time" + "unsafe" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -57,12 +58,18 @@ type LazyTxn struct { stagingHandle kv.StagingHandle mutations map[int64]*binlog.TableMutation writeSLI sli.TxnWriteThroughputSLI + + // following atomic fields are used for filling TxnInfo + // we need these fields because kv.Transaction provides no thread safety promise + // but we hope getting TxnInfo is a thread safe op + + infoStartTS uint64 // digest of SQL current running - CurrentSQLDigest string + CurrentSQLDigest atomic.Value // string // current executing state State txnInfo.TxnRunningState // last trying to block start time - blockStartTime *time.Time + blockStartTime unsafe.Pointer // *time.Time, cannot use atomic.Value here because it is possible to be nil } // GetTableInfo returns the cached index name. @@ -77,7 +84,7 @@ func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) - txn.State = txnInfo.TxnRunningNormal + atomic.StoreInt32(&txn.State, txnInfo.TxnRunningNormal) } func (txn *LazyTxn) initStmtBuf() { @@ -168,7 +175,8 @@ func (txn *LazyTxn) GoString() string { func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { txn.Transaction = kvTxn - txn.State = txnInfo.TxnRunningNormal + atomic.StoreInt32(&txn.State, txnInfo.TxnRunningNormal) + atomic.StoreUint64(&txn.infoStartTS, kvTxn.StartTS()) txn.initStmtBuf() txn.txnFuture = nil } @@ -176,6 +184,7 @@ func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { func (txn *LazyTxn) changeInvalidToPending(future *txnFuture) { txn.Transaction = nil txn.txnFuture = future + atomic.StoreUint64(&txn.infoStartTS, 0) } func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { @@ -193,7 +202,8 @@ func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t - txn.State = txnInfo.TxnRunningNormal + atomic.StoreInt32(&txn.State, txnInfo.TxnRunningNormal) + atomic.StoreUint64(&txn.infoStartTS, t.StartTS()) txn.initStmtBuf() return nil } @@ -205,6 +215,7 @@ func (txn *LazyTxn) changeToInvalid() { txn.stagingHandle = kv.InvalidStagingHandle txn.Transaction = nil txn.txnFuture = nil + atomic.StoreUint64(&txn.infoStartTS, 0) } var hasMockAutoIncIDRetry = int64(0) @@ -244,7 +255,7 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } - txn.State = txnInfo.TxnCommitting + atomic.StoreInt32(&txn.State, txnInfo.TxnCommitting) // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { @@ -280,12 +291,13 @@ func (txn *LazyTxn) Rollback() error { // LockKeys Wrap the inner transaction's `LockKeys` to record the status func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { - originState := txn.State - txn.State = txnInfo.TxnLockWaiting + originState := atomic.LoadInt32(&txn.State) + atomic.StoreInt32(&txn.State, txnInfo.TxnLockWaiting) t := time.Now() - txn.blockStartTime = &t + atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(&t)) err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) - txn.blockStartTime = nil + atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(nil)) + atomic.StoreInt32(&txn.State, originState) txn.State = originState return err } @@ -343,17 +355,19 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { } // Info dump the TxnState to Datum for displaying in `TIDB_TRX` +// This function is supposed to be thread safe func (txn *LazyTxn) Info() *txnInfo.TxnInfo { - if !txn.Valid() { + startTs := atomic.LoadUint64(&txn.infoStartTS) + if startTs == 0 { return nil } return &txnInfo.TxnInfo{ - StartTS: txn.StartTS(), - CurrentSQLDigest: txn.CurrentSQLDigest, - State: txn.State, - BlockStartTime: txn.blockStartTime, - Len: txn.Transaction.Len(), - Size: txn.Transaction.Size(), + StartTS: startTs, + CurrentSQLDigest: *txn.CurrentSQLDigest.Load().(*string), + State: atomic.LoadInt32(&txn.State), + BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), + Len: int64(txn.Len()), + Size: int64(txn.Size()), } } diff --git a/session/txnInfo/txnInfo.go b/session/txnInfo/txnInfo.go index d1f79d8254a82..fcf248b74afc7 100644 --- a/session/txnInfo/txnInfo.go +++ b/session/txnInfo/txnInfo.go @@ -22,7 +22,7 @@ import ( ) // TxnRunningState is the current state of a transaction -type TxnRunningState = int +type TxnRunningState = int32 const ( // TxnRunningNormal means the transaction is running normally @@ -46,9 +46,9 @@ type TxnInfo struct { // last trying to block start time BlockStartTime *time.Time // How many entries are in MemDB - Len int + Len int64 // MemDB used memory - Size int + Size int64 // the following fields will be filled in `session` instead of `LazyTxn` diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index a008a7cc6e0ab..d00730d9f70cb 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -18,6 +18,7 @@ import ( "math" "reflect" "sync" + "sync/atomic" "unsafe" tidbkv "github.com/pingcap/tidb/kv" @@ -60,8 +61,8 @@ type MemDB struct { entrySizeLimit uint64 bufferSizeLimit uint64 - count int - size int + count int64 + size int64 vlogInvalid bool dirty bool @@ -260,12 +261,12 @@ func (db *MemDB) GetValueByHandle(handle MemKeyHandle) ([]byte, bool) { // Len returns the number of entries in the DB. func (db *MemDB) Len() int { - return db.count + return int(atomic.LoadInt64(&db.count)) } // Size returns sum of keys and values length. func (db *MemDB) Size() int { - return db.size + return int(atomic.LoadInt64(&db.size)) } // Dirty returns whether the root staging buffer is updated. @@ -332,7 +333,7 @@ func (db *MemDB) setValue(x memdbNodeAddr, value []byte) { } } x.vptr = db.vlog.appendValue(x.addr, x.vptr, value) - db.size = db.size - len(oldVal) + len(value) + atomic.AddInt64(&db.size, int64(len(value)-len(oldVal))) } // traverse search for and if not found and insert is true, will add a new node in. @@ -536,8 +537,8 @@ func (db *MemDB) rightRotate(y memdbNodeAddr) { func (db *MemDB) deleteNode(z memdbNodeAddr) { var x, y memdbNodeAddr - db.count-- - db.size -= int(z.klen) + atomic.AddInt64(&db.count, -1) + atomic.AddInt64(&db.size, -int64(z.klen)) if z.left.isNull() || z.right.isNull() { y = z @@ -736,8 +737,8 @@ func (db *MemDB) getRoot() memdbNodeAddr { } func (db *MemDB) allocNode(key []byte) memdbNodeAddr { - db.size += len(key) - db.count++ + atomic.AddInt64(&db.size, int64(len(key))) + atomic.AddInt64(&db.count, int64(1)) x, xn := db.allocator.allocNode(key) return memdbNodeAddr{xn, x} } diff --git a/store/tikv/unionstore/memdb_arena.go b/store/tikv/unionstore/memdb_arena.go index 45f7f6279e2e5..12818e2282833 100644 --- a/store/tikv/unionstore/memdb_arena.go +++ b/store/tikv/unionstore/memdb_arena.go @@ -16,6 +16,7 @@ package unionstore import ( "encoding/binary" "math" + "sync/atomic" "unsafe" "github.com/pingcap/tidb/store/tikv/kv" @@ -309,7 +310,7 @@ func (l *memdbVlog) revertToCheckpoint(db *MemDB, cp *memdbCheckpoint) { node := db.getNode(hdr.nodeAddr) node.vptr = hdr.oldValue - db.size -= int(hdr.valueLen) + atomic.AddInt64(&db.size, -int64(hdr.valueLen)) // oldValue.isNull() == true means this is a newly added value. if hdr.oldValue.isNull() { // If there are no flags associated with this key, we need to delete this node. @@ -321,7 +322,7 @@ func (l *memdbVlog) revertToCheckpoint(db *MemDB, cp *memdbCheckpoint) { db.dirty = true } } else { - db.size += len(l.getValue(hdr.oldValue)) + atomic.AddInt64(&db.size, int64(len(l.getValue(hdr.oldValue)))) } l.moveBackCursor(&cursor, &hdr) From 355f0f7f21a3bfb024fb41856dd6380b14d2986f Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 6 May 2021 21:37:43 +0800 Subject: [PATCH 17/26] use enum to display TxnRunningState --- infoschema/tables.go | 23 +++++++++++++---------- session/txnInfo/txnInfo.go | 23 +++++++++++++++++------ util/chunk/column.go | 6 +++++- 3 files changed, 35 insertions(+), 17 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 2a81a2eb8d7f1..9d036aace77d5 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/session/txnInfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" @@ -241,19 +242,20 @@ var tableIDMap = map[string]int64{ } type columnInfo struct { - name string - tp byte - size int - decimal int - flag uint - deflt interface{} - comment string + name string + tp byte + size int + decimal int + flag uint + deflt interface{} + comment string + enumElems []string } func buildColumnInfo(col columnInfo) *model.ColumnInfo { mCharset := charset.CharsetBin mCollation := charset.CharsetBin - if col.tp == mysql.TypeVarchar || col.tp == mysql.TypeBlob || col.tp == mysql.TypeLongBlob { + if col.tp == mysql.TypeVarchar || col.tp == mysql.TypeBlob || col.tp == mysql.TypeLongBlob || col.tp == mysql.TypeEnum { mCharset = charset.CharsetUTF8MB4 mCollation = charset.CollationUTF8MB4 } @@ -264,6 +266,7 @@ func buildColumnInfo(col columnInfo) *model.ColumnInfo { Flen: col.size, Decimal: col.decimal, Flag: col.flag, + Elems: col.enumElems, } return &model.ColumnInfo{ Name: model.NewCIStr(col.name), @@ -1341,11 +1344,11 @@ var tableTiDBTrxCols = []columnInfo{ {name: "ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, - {name: "STATE", tp: mysql.TypeLonglong, size: 64, comment: "Current running state of the transaction"}, + {name: "STATE", tp: mysql.TypeEnum, enumElems: txnInfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, - {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, + {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, } diff --git a/session/txnInfo/txnInfo.go b/session/txnInfo/txnInfo.go index fcf248b74afc7..3a90f28008ca1 100644 --- a/session/txnInfo/txnInfo.go +++ b/session/txnInfo/txnInfo.go @@ -28,13 +28,17 @@ const ( // TxnRunningNormal means the transaction is running normally TxnRunningNormal TxnRunningState = iota // TxnLockWaiting means the transaction is blocked on a lock - TxnLockWaiting TxnRunningState = iota + TxnLockWaiting // TxnCommitting means the transaction is (at least trying to) committing - TxnCommitting TxnRunningState = iota + TxnCommitting // TxnRollingBack means the transaction is rolling back - TxnRollingBack TxnRunningState = iota + TxnRollingBack ) +var TxnRunningStateStrs = []string{ + "Normal", "LockWaiting", "Committing", "RollingBack", +} + // TxnInfo is information about a running transaction // This is supposed to be the datasource of `TIDB_TRX` in infoschema type TxnInfo struct { @@ -69,13 +73,20 @@ func (info TxnInfo) ToDatum() []types.Datum { } else { blockStartTime = types.NewTime(types.FromGoTime(*info.BlockStartTime), mysql.TypeTimestamp, 0) } - return types.MakeDatums( + e, _ := types.ParseEnumValue(TxnRunningStateStrs, uint64(info.State+1)) + state := types.NewMysqlEnumDatum(e) + datums := types.MakeDatums( info.StartTS, types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), info.CurrentSQLDigest, - info.State, + ) + datums = append(datums, state) + datums = append(datums, types.MakeDatums( blockStartTime, info.Len, info.Size, - ) + info.ConnectionID, + info.Username, + info.CurrentDB)...) + return datums } diff --git a/util/chunk/column.go b/util/chunk/column.go index 92ac8e3afd1d2..3778bf748469d 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -20,9 +20,11 @@ import ( "time" "unsafe" + "github.com/pingcap/log" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/hack" + "go.uber.org/zap" ) // AppendDuration appends a duration value into this Column. @@ -569,7 +571,9 @@ func (c *Column) getNameValue(rowID int) (string, uint64) { } var val uint64 copy((*[8]byte)(unsafe.Pointer(&val))[:], c.data[start:]) - return string(hack.String(c.data[start+8 : end])), val + s := string(hack.String(c.data[start+8 : end])) + log.Info("getNameValue", zap.String("s", s), zap.Uint64("val", val)) + return s, val } // GetRaw returns the underlying raw bytes in the specific row. From a4952561fa28fc04de9b7d6bf04993972de4743f Mon Sep 17 00:00:00 2001 From: longfangsong Date: Thu, 6 May 2021 21:54:55 +0800 Subject: [PATCH 18/26] Add Privilege check --- executor/infoschema_reader.go | 20 ++++++++++++++++---- infoschema/tables.go | 2 +- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index b54072a803f1c..0106e931223b3 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -151,7 +151,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) case infoschema.TableTiDBTrx, infoschema.ClusterTableTiDBTrx: - err = e.setDataForTiDBTrx(sctx) + e.setDataForTiDBTrx(sctx) } if err != nil { return nil, err @@ -2014,16 +2014,28 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context return nil } -func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) error { +func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) { sm := ctx.GetSessionManager() if sm == nil { - return nil + return + } + + loginUser := ctx.GetSessionVars().User + var hasProcessPriv bool + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { + hasProcessPriv = true + } } infoList := sm.ShowTxnList() for _, info := range infoList { + // If you have the PROCESS privilege, you can see all running transactions. + // Otherwise, you can see only your own transactions. + if !hasProcessPriv && loginUser != nil && info.Username != loginUser.Username { + continue + } e.rows = append(e.rows, info.ToDatum()) } - return nil } type hugeMemTableRetriever struct { diff --git a/infoschema/tables.go b/infoschema/tables.go index 9d036aace77d5..5cf982ae81072 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1341,7 +1341,7 @@ var tableClientErrorsSummaryByHostCols = []columnInfo{ } var tableTiDBTrxCols = []columnInfo{ - {name: "ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, + {name: "ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeEnum, enumElems: txnInfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, From c3aad93a7c6043cf61e647f6bb37cb6238afa843 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 6 May 2021 21:55:50 +0800 Subject: [PATCH 19/26] Update server/server.go Co-authored-by: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> --- server/server.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/server.go b/server/server.go index 3e5b60f83865c..86cf55b326d41 100644 --- a/server/server.go +++ b/server/server.go @@ -562,7 +562,7 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { func (s *Server) ShowTxnList() []txnInfo.TxnInfo { s.rwlock.RLock() defer s.rwlock.RUnlock() - rs := []txnInfo.TxnInfo{} + rs := make([]txnInfo.TxnInfo, 0, len(s.clients)) for _, client := range s.clients { if client.ctx.Session != nil { info := client.ctx.Session.TxnInfo() From 4d504db8ac77c4f885f8bfaae8bdbbd0c5bb2a60 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Fri, 7 May 2021 00:14:09 +0800 Subject: [PATCH 20/26] address some comments --- bindinfo/bind_test.go | 4 +-- domain/domain_test.go | 4 +-- executor/executor_pkg_test.go | 4 +-- executor/explainfor_test.go | 4 +-- executor/infoschema_reader_test.go | 4 +-- executor/prepared_test.go | 4 +-- executor/seqtest/prepared_test.go | 4 +-- infoschema/tables.go | 4 +-- infoschema/tables_test.go | 4 +-- server/server.go | 8 ++--- session/session.go | 12 +++---- session/session_test.go | 14 ++++---- session/txn.go | 33 +++++++++---------- .../txnInfo.go => txninfo/txn_info.go} | 10 +++--- store/tikv/unionstore/memdb.go | 4 +-- util/processinfo.go | 4 +-- 16 files changed, 57 insertions(+), 64 deletions(-) rename session/{txnInfo/txnInfo.go => txninfo/txn_info.go} (96%) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 443ae04cfcbe0..22b60187f0a77 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/util" @@ -71,7 +71,7 @@ type mockSessionManager struct { PS []*util.ProcessInfo } -func (msm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/domain/domain_test.go b/domain/domain_test.go index afd4bf95fc532..a4432b0fb1fe6 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" @@ -242,7 +242,7 @@ type mockSessionManager struct { PS []*util.ProcessInfo } -func (msm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 39ea2f47e1833..5591dcefde54d 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" plannerutil "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/session/txnInfo" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -61,7 +61,7 @@ type mockSessionManager struct { serverID uint64 } -func (msm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 5a9a3df422738..e29a7a3e24cee 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" @@ -39,7 +39,7 @@ type mockSessionManager1 struct { PS []*util.ProcessInfo } -func (msm *mockSessionManager1) ShowTxnList() []txnInfo.TxnInfo { +func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo { return nil } diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 8b8e0fa973fb9..e19eb9d9b3064 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" @@ -729,7 +729,7 @@ type mockSessionManager struct { serverID uint64 } -func (sm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { +func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/executor/prepared_test.go b/executor/prepared_test.go index ee0d8e6922c13..e0e2c19ee0f22 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/domain" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" @@ -136,7 +136,7 @@ type mockSessionManager2 struct { killed bool } -func (sm *mockSessionManager2) ShowTxnList() []txnInfo.TxnInfo { +func (sm *mockSessionManager2) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 7ab408eed754f..bb8f05e5eff54 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" @@ -797,7 +797,7 @@ type mockSessionManager1 struct { Se session.Session } -func (msm *mockSessionManager1) ShowTxnList() []txnInfo.TxnInfo { +func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/infoschema/tables.go b/infoschema/tables.go index 5cf982ae81072..03587ab046adc 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -37,7 +37,7 @@ import ( "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" @@ -1344,7 +1344,7 @@ var tableTiDBTrxCols = []columnInfo{ {name: "ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, - {name: "STATE", tp: mysql.TypeEnum, enumElems: txnInfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, + {name: "STATE", tp: mysql.TypeEnum, enumElems: txninfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 5dcf18338d86d..a03ec3021db75 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -42,7 +42,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" @@ -436,7 +436,7 @@ type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo } -func (sm *mockSessionManager) ShowTxnList() []txnInfo.TxnInfo { +func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { panic("unimplemented!") } diff --git a/server/server.go b/server/server.go index 86cf55b326d41..29f5307895cc2 100644 --- a/server/server.go +++ b/server/server.go @@ -56,7 +56,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/plugin" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util" @@ -559,15 +559,15 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { } // ShowTxnList shows all txn info for displaying in `TIDB_TRX` -func (s *Server) ShowTxnList() []txnInfo.TxnInfo { +func (s *Server) ShowTxnList() []*txninfo.TxnInfo { s.rwlock.RLock() defer s.rwlock.RUnlock() - rs := make([]txnInfo.TxnInfo, 0, len(s.clients)) + rs := make([]*txninfo.TxnInfo, 0, len(s.clients)) for _, client := range s.clients { if client.ctx.Session != nil { info := client.ctx.Session.TxnInfo() if info != nil { - rs = append(rs, *info) + rs = append(rs, info) } } } diff --git a/session/session.go b/session/session.go index 5159e295bfa55..d11b58103b007 100644 --- a/session/session.go +++ b/session/session.go @@ -61,7 +61,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" - "github.com/pingcap/tidb/session/txnInfo" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -148,7 +148,7 @@ type Session interface { AuthWithoutVerification(user *auth.UserIdentity) bool ShowProcess() *util.ProcessInfo // Return the information of the txn current running - TxnInfo() *txnInfo.TxnInfo + TxnInfo() *txninfo.TxnInfo // PrepareTxnCtx is exported for test. PrepareTxnCtx(context.Context) // FieldList returns fields list of a table. @@ -446,12 +446,13 @@ func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { return fields, nil } -func (s *session) TxnInfo() *txnInfo.TxnInfo { +func (s *session) TxnInfo() *txninfo.TxnInfo { txnInfo := s.txn.Info() if txnInfo == nil { return nil } processInfo := s.ShowProcess() + txnInfo.CurrentSQLDigest = processInfo.Digest txnInfo.ConnectionID = processInfo.ID txnInfo.Username = processInfo.User txnInfo.CurrentDB = processInfo.DB @@ -1434,11 +1435,6 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, } func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { - if s.txn.Valid() { - _, digest := parser.NormalizeDigest(stmtNode.Text()) - s.txn.CurrentSQLDigest.Store(&digest) - } - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.ExecuteStmt", opentracing.ChildOf(span.Context())) defer span1.Finish() diff --git a/session/session_test.go b/session/session_test.go index f04947d87a4f8..dfd7889713257 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -42,7 +42,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/session/txnInfo" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" @@ -4282,7 +4282,7 @@ func (s *testTxnStateSuite) TestBasic(c *C) { info = tk.Se.TxnInfo() _, expectedDigest := parser.NormalizeDigest("select * from t for update;") c.Assert(info.CurrentSQLDigest, Equals, expectedDigest) - c.Assert(info.State, Equals, txnInfo.TxnRunningNormal) + c.Assert(info.State, Equals, txninfo.TxnRunningNormal) c.Assert(info.BlockStartTime, IsNil) // len and size will be covered in TestLenAndSize c.Assert(info.ConnectionID, Equals, tk.Se.GetSessionVars().ConnectionID) @@ -4306,7 +4306,7 @@ func (s *testTxnStateSuite) TestBlocked(c *C) { tk2.MustExec("commit;") }() time.Sleep(200 * time.Millisecond) - c.Assert(tk2.Se.TxnInfo().State, Equals, txnInfo.TxnLockWaiting) + c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnLockWaiting) c.Assert(tk2.Se.TxnInfo().BlockStartTime, NotNil) tk.MustExec("commit;") } @@ -4317,11 +4317,11 @@ func (s *testTxnStateSuite) TestLenAndSize(c *C) { tk.MustExec("begin pessimistic;") tk.MustExec("insert into t(a) values (1);") info := tk.Se.TxnInfo() - c.Assert(info.Len, Equals, int64(1)) - c.Assert(info.Size, Equals, int64(29)) + c.Assert(info.EntriesCount, Equals, int64(1)) + c.Assert(info.EntriesSize, Equals, int64(29)) tk.MustExec("insert into t(a) values (2);") info = tk.Se.TxnInfo() - c.Assert(info.Len, Equals, int64(2)) - c.Assert(info.Size, Equals, int64(58)) + c.Assert(info.EntriesCount, Equals, int64(2)) + c.Assert(info.EntriesSize, Equals, int64(58)) tk.MustExec("commit;") } diff --git a/session/txn.go b/session/txn.go index ba2d73d47aa4a..ae46305ef1021 100644 --- a/session/txn.go +++ b/session/txn.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -64,10 +64,8 @@ type LazyTxn struct { // but we hope getting TxnInfo is a thread safe op infoStartTS uint64 - // digest of SQL current running - CurrentSQLDigest atomic.Value // string // current executing state - State txnInfo.TxnRunningState + State txninfo.TxnRunningState // last trying to block start time blockStartTime unsafe.Pointer // *time.Time, cannot use atomic.Value here because it is possible to be nil } @@ -84,7 +82,7 @@ func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) - atomic.StoreInt32(&txn.State, txnInfo.TxnRunningNormal) + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) } func (txn *LazyTxn) initStmtBuf() { @@ -175,7 +173,7 @@ func (txn *LazyTxn) GoString() string { func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { txn.Transaction = kvTxn - atomic.StoreInt32(&txn.State, txnInfo.TxnRunningNormal) + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) atomic.StoreUint64(&txn.infoStartTS, kvTxn.StartTS()) txn.initStmtBuf() txn.txnFuture = nil @@ -202,7 +200,7 @@ func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t - atomic.StoreInt32(&txn.State, txnInfo.TxnRunningNormal) + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) atomic.StoreUint64(&txn.infoStartTS, t.StartTS()) txn.initStmtBuf() return nil @@ -255,7 +253,7 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } - atomic.StoreInt32(&txn.State, txnInfo.TxnCommitting) + atomic.StoreInt32(&txn.State, txninfo.TxnCommitting) // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { @@ -285,14 +283,14 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { // Rollback overrides the Transaction interface. func (txn *LazyTxn) Rollback() error { defer txn.reset() - txn.State = txnInfo.TxnRollingBack + txn.State = txninfo.TxnRollingBack return txn.Transaction.Rollback() } // LockKeys Wrap the inner transaction's `LockKeys` to record the status func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { originState := atomic.LoadInt32(&txn.State) - atomic.StoreInt32(&txn.State, txnInfo.TxnLockWaiting) + atomic.StoreInt32(&txn.State, txninfo.TxnLockWaiting) t := time.Now() atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(&t)) err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) @@ -356,18 +354,17 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { // Info dump the TxnState to Datum for displaying in `TIDB_TRX` // This function is supposed to be thread safe -func (txn *LazyTxn) Info() *txnInfo.TxnInfo { +func (txn *LazyTxn) Info() *txninfo.TxnInfo { startTs := atomic.LoadUint64(&txn.infoStartTS) if startTs == 0 { return nil } - return &txnInfo.TxnInfo{ - StartTS: startTs, - CurrentSQLDigest: *txn.CurrentSQLDigest.Load().(*string), - State: atomic.LoadInt32(&txn.State), - BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), - Len: int64(txn.Len()), - Size: int64(txn.Size()), + return &txninfo.TxnInfo{ + StartTS: startTs, + State: atomic.LoadInt32(&txn.State), + BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), + EntriesCount: int64(txn.Len()), + EntriesSize: int64(txn.Size()), } } diff --git a/session/txnInfo/txnInfo.go b/session/txninfo/txn_info.go similarity index 96% rename from session/txnInfo/txnInfo.go rename to session/txninfo/txn_info.go index 3a90f28008ca1..a3c3064a949ad 100644 --- a/session/txnInfo/txnInfo.go +++ b/session/txninfo/txn_info.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package txnInfo +package txninfo import ( "time" @@ -50,9 +50,9 @@ type TxnInfo struct { // last trying to block start time BlockStartTime *time.Time // How many entries are in MemDB - Len int64 + EntriesCount int64 // MemDB used memory - Size int64 + EntriesSize int64 // the following fields will be filled in `session` instead of `LazyTxn` @@ -83,8 +83,8 @@ func (info TxnInfo) ToDatum() []types.Datum { datums = append(datums, state) datums = append(datums, types.MakeDatums( blockStartTime, - info.Len, - info.Size, + info.EntriesCount, + info.EntriesSize, info.ConnectionID, info.Username, info.CurrentDB)...) diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index 149a93a8dce31..51db9d2fbf946 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -139,8 +139,8 @@ func (db *MemDB) Reset() { db.stages = db.stages[:0] db.dirty = false db.vlogInvalid = false - db.size = 0 - db.count = 0 + atomic.StoreInt64(&db.size, 0) + atomic.StoreInt64(&db.count, 0) db.vlog.reset() db.allocator.reset() } diff --git a/util/processinfo.go b/util/processinfo.go index 5daeb96e04639..ebbf17094b80d 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -22,7 +22,7 @@ import ( "time" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/session/txnInfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/execdetails" @@ -162,7 +162,7 @@ func serverStatus2Str(state uint16) string { // kill statement rely on this interface. type SessionManager interface { ShowProcessList() map[uint64]*ProcessInfo - ShowTxnList() []txnInfo.TxnInfo + ShowTxnList() []*txninfo.TxnInfo GetProcessInfo(id uint64) (*ProcessInfo, bool) Kill(connectionID uint64, query bool) KillAllConnections() From 065667bb986fe50a3e52584efcb3701f0480f403 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Fri, 7 May 2021 13:16:07 +0800 Subject: [PATCH 21/26] apply some suggestions from reviewing --- executor/infoschema_reader.go | 15 +++++++++++++-- infoschema/infoschema_test.go | 1 + session/txninfo/txn_info.go | 6 +++++- 3 files changed, 19 insertions(+), 3 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 0106e931223b3..5cc5658ee076d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -149,9 +149,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex infoschema.TableClientErrorsSummaryByUser, infoschema.TableClientErrorsSummaryByHost: err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) - case infoschema.TableTiDBTrx, - infoschema.ClusterTableTiDBTrx: + case infoschema.TableTiDBTrx: e.setDataForTiDBTrx(sctx) + case infoschema.ClusterTableTiDBTrx: + err = e.setDataForClusterTiDBTrx(sctx) } if err != nil { return nil, err @@ -2038,6 +2039,16 @@ func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) { } } +func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) error { + e.setDataForTiDBTrx(ctx) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) + if err != nil { + return err + } + e.rows = rows + return nil +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index c3892e6527962..6aa0c5526f467 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -332,6 +332,7 @@ func (*testSuite) TestInfoTables(c *C) { "TABLESPACES", "COLLATION_CHARACTER_SET_APPLICABILITY", "PROCESSLIST", + "TIDB_TRX", } for _, t := range infoTables { tb, err1 := is.TableByName(util.InformationSchemaName, model.NewCIStr(t)) diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index a3c3064a949ad..3bbc3aa19b46f 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -35,6 +35,7 @@ const ( TxnRollingBack ) +// TxnRunningStateStrs is the names of the TxnRunningStates var TxnRunningStateStrs = []string{ "Normal", "LockWaiting", "Committing", "RollingBack", } @@ -73,7 +74,10 @@ func (info TxnInfo) ToDatum() []types.Datum { } else { blockStartTime = types.NewTime(types.FromGoTime(*info.BlockStartTime), mysql.TypeTimestamp, 0) } - e, _ := types.ParseEnumValue(TxnRunningStateStrs, uint64(info.State+1)) + e, err := types.ParseEnumValue(TxnRunningStateStrs, uint64(info.State+1)) + if err != nil { + panic("this should never happen") + } state := types.NewMysqlEnumDatum(e) datums := types.MakeDatums( info.StartTS, From fccbf3db8e1cf12be73392fc2f0d0a345f30f1a0 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Fri, 7 May 2021 15:14:49 +0800 Subject: [PATCH 22/26] add test for querying with SQL --- infoschema/tables_test.go | 33 ++++++++++++++++++++++++++++----- util/chunk/column.go | 6 +----- 2 files changed, 29 insertions(+), 10 deletions(-) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index a03ec3021db75..6cc24300c1be4 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -28,6 +28,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/fn" + "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -122,7 +123,7 @@ func (s *testClusterTableSuite) setUpRPCService(c *C, addr string) (*grpc.Server lis, err := net.Listen("tcp", addr) c.Assert(err, IsNil) // Fix issue 9836 - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "root", @@ -277,7 +278,7 @@ func (s *testTableSuite) TestInfoschemaFieldValue(c *C) { tk1.MustQuery("select distinct(table_schema) from information_schema.tables").Check(testkit.Rows("INFORMATION_SCHEMA")) // Fix issue 9836 - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "root", @@ -434,10 +435,11 @@ func (s *testTableSuite) TestCurrentTimestampAsDefault(c *C) { type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo + txnInfo []*txninfo.TxnInfo } func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { - panic("unimplemented!") + return sm.txnInfo } func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { @@ -464,7 +466,7 @@ func (s *testTableSuite) TestSomeTables(c *C) { c.Assert(err, IsNil) tk := testkit.NewTestKit(c, s.store) tk.Se = se - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "user-1", @@ -521,7 +523,7 @@ func (s *testTableSuite) TestSomeTables(c *C) { fmt.Sprintf("3 user-3 127.0.0.1:12345 test Init DB 9223372036 %s %s", "in transaction", "check port"), )) - sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "user-1", @@ -1514,3 +1516,24 @@ func (s *testTableSuite) TestInfoschemaClientErrors(c *C) { err = tk.ExecToErr("FLUSH CLIENT_ERRORS_SUMMARY") c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RELOAD privilege(s) for this operation") } + +func (s *testTableSuite) TestTrx(c *C) { + tk := s.newTestKitWithRoot(c) + _, digest := parser.NormalizeDigest("select * from trx for update;") + sm := &mockSessionManager{nil, make([]*txninfo.TxnInfo, 1)} + sm.txnInfo[0] = &txninfo.TxnInfo{ + StartTS: 424768545227014155, + CurrentSQLDigest: digest, + State: txninfo.TxnRunningNormal, + BlockStartTime: nil, + EntriesCount: 1, + EntriesSize: 19, + ConnectionID: 2, + Username: "root", + CurrentDB: "test", + } + tk.Se.SetSessionManager(sm) + tk.MustQuery("select * from information_schema.TIDB_TRX;").Check( + testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest + " Normal 1 19 2 root test"), + ) +} diff --git a/util/chunk/column.go b/util/chunk/column.go index 3778bf748469d..92ac8e3afd1d2 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -20,11 +20,9 @@ import ( "time" "unsafe" - "github.com/pingcap/log" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/hack" - "go.uber.org/zap" ) // AppendDuration appends a duration value into this Column. @@ -571,9 +569,7 @@ func (c *Column) getNameValue(rowID int) (string, uint64) { } var val uint64 copy((*[8]byte)(unsafe.Pointer(&val))[:], c.data[start:]) - s := string(hack.String(c.data[start+8 : end])) - log.Info("getNameValue", zap.String("s", s), zap.Uint64("val", val)) - return s, val + return string(hack.String(c.data[start+8 : end])), val } // GetRaw returns the underlying raw bytes in the specific row. From 20109e773707539a4a7db587f277dce49558be57 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Sat, 8 May 2021 15:20:50 +0800 Subject: [PATCH 23/26] Add tests for TxnCommitting and TxnRollingBack --- session/session_test.go | 53 +++++++++++++++++++++++++++++++++-------- session/txn.go | 4 ++++ 2 files changed, 47 insertions(+), 10 deletions(-) diff --git a/session/session_test.go b/session/session_test.go index dfd7889713257..bf60cebe01e89 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4293,6 +4293,21 @@ func (s *testTxnStateSuite) TestBasic(c *C) { c.Assert(info, IsNil) } +func (s *testTxnStateSuite) TestEntriesCountAndSize(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("begin pessimistic;") + tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info.EntriesCount, Equals, int64(1)) + c.Assert(info.EntriesSize, Equals, int64(29)) + tk.MustExec("insert into t(a) values (2);") + info = tk.Se.TxnInfo() + c.Assert(info.EntriesCount, Equals, int64(2)) + c.Assert(info.EntriesSize, Equals, int64(58)) + tk.MustExec("commit;") +} + func (s *testTxnStateSuite) TestBlocked(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) @@ -4305,23 +4320,41 @@ func (s *testTxnStateSuite) TestBlocked(c *C) { tk2.MustExec("select * from t where a = 1 for update;") tk2.MustExec("commit;") }() - time.Sleep(200 * time.Millisecond) + time.Sleep(100 * time.Millisecond) c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnLockWaiting) c.Assert(tk2.Se.TxnInfo().BlockStartTime, NotNil) tk.MustExec("commit;") } -func (s *testTxnStateSuite) TestLenAndSize(c *C) { +func (s *testTxnStateSuite) TestCommitting(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1), (2);") tk.MustExec("begin pessimistic;") - tk.MustExec("insert into t(a) values (1);") - info := tk.Se.TxnInfo() - c.Assert(info.EntriesCount, Equals, int64(1)) - c.Assert(info.EntriesSize, Equals, int64(29)) - tk.MustExec("insert into t(a) values (2);") - info = tk.Se.TxnInfo() - c.Assert(info.EntriesCount, Equals, int64(2)) - c.Assert(info.EntriesSize, Equals, int64(58)) + tk.MustExec("select * from t where a = 1 for update;") + go func() { + tk2.MustExec("begin pessimistic") + c.Assert(tk2.Se.TxnInfo(), NotNil) + tk2.MustExec("select * from t where a = 2 for update;") + failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)") + tk2.MustExec("commit;") + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnCommitting) tk.MustExec("commit;") } + +func (s *testTxnStateSuite) TestRollbacking(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1), (2);") + go func() { + tk.MustExec("begin pessimistic") + tk.MustExec("insert into t(a) values (3);") + failpoint.Enable("github.com/pingcap/tidb/session/mockSlowRollback", "sleep(200)") + tk.MustExec("rollback;") + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk.Se.TxnInfo().State, Equals, txninfo.TxnRollingBack) +} diff --git a/session/txn.go b/session/txn.go index ae46305ef1021..362e5333dd007 100644 --- a/session/txn.go +++ b/session/txn.go @@ -255,6 +255,8 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { atomic.StoreInt32(&txn.State, txninfo.TxnCommitting) + failpoint.Inject("mockSlowCommit", func(_ failpoint.Value) {}) + // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { if val.(bool) { @@ -284,6 +286,8 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { func (txn *LazyTxn) Rollback() error { defer txn.reset() txn.State = txninfo.TxnRollingBack + // mockSlowRollback is used to mock a rollback which takes a long time + failpoint.Inject("mockSlowRollback", func(_ failpoint.Value) {}) return txn.Transaction.Rollback() } From f69329801f4d20ee2605a7ff725d5ac4ee406f08 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Sat, 8 May 2021 16:12:16 +0800 Subject: [PATCH 24/26] disable failpoint after use --- session/session_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/session/session_test.go b/session/session_test.go index 608e886fd222c..550aea75d5e7e 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4375,6 +4375,7 @@ func (s *testTxnStateSuite) TestCommitting(c *C) { c.Assert(tk2.Se.TxnInfo(), NotNil) tk2.MustExec("select * from t where a = 2 for update;") failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)") + defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowCommit") tk2.MustExec("commit;") }() time.Sleep(100 * time.Millisecond) @@ -4390,6 +4391,7 @@ func (s *testTxnStateSuite) TestRollbacking(c *C) { tk.MustExec("begin pessimistic") tk.MustExec("insert into t(a) values (3);") failpoint.Enable("github.com/pingcap/tidb/session/mockSlowRollback", "sleep(200)") + defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowRollback") tk.MustExec("rollback;") }() time.Sleep(100 * time.Millisecond) From 68edd438cdcec7adffcbeeabaa4cf864f3ce8455 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Mon, 10 May 2021 10:24:06 +0800 Subject: [PATCH 25/26] Address comments --- session/session_test.go | 6 ++++++ session/txn.go | 3 +-- session/txninfo/txn_info.go | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/session/session_test.go b/session/session_test.go index 550aea75d5e7e..b54a22660ea2f 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4370,6 +4370,7 @@ func (s *testTxnStateSuite) TestCommitting(c *C) { tk.MustExec("insert into t(a) values (1), (2);") tk.MustExec("begin pessimistic;") tk.MustExec("select * from t where a = 1 for update;") + ch := make(chan struct{}) go func() { tk2.MustExec("begin pessimistic") c.Assert(tk2.Se.TxnInfo(), NotNil) @@ -4377,23 +4378,28 @@ func (s *testTxnStateSuite) TestCommitting(c *C) { failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)") defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowCommit") tk2.MustExec("commit;") + ch <- struct{}{} }() time.Sleep(100 * time.Millisecond) c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnCommitting) tk.MustExec("commit;") + <-ch } func (s *testTxnStateSuite) TestRollbacking(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") tk.MustExec("insert into t(a) values (1), (2);") + ch := make(chan struct{}) go func() { tk.MustExec("begin pessimistic") tk.MustExec("insert into t(a) values (3);") failpoint.Enable("github.com/pingcap/tidb/session/mockSlowRollback", "sleep(200)") defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowRollback") tk.MustExec("rollback;") + ch <- struct{}{} }() time.Sleep(100 * time.Millisecond) c.Assert(tk.Se.TxnInfo().State, Equals, txninfo.TxnRollingBack) + <-ch } diff --git a/session/txn.go b/session/txn.go index 362e5333dd007..0c1a1dc4d4741 100644 --- a/session/txn.go +++ b/session/txn.go @@ -285,7 +285,7 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { // Rollback overrides the Transaction interface. func (txn *LazyTxn) Rollback() error { defer txn.reset() - txn.State = txninfo.TxnRollingBack + atomic.StoreInt32(&txn.State, txninfo.TxnRollingBack) // mockSlowRollback is used to mock a rollback which takes a long time failpoint.Inject("mockSlowRollback", func(_ failpoint.Value) {}) return txn.Transaction.Rollback() @@ -300,7 +300,6 @@ func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...k err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(nil)) atomic.StoreInt32(&txn.State, originState) - txn.State = originState return err } diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index 3bbc3aa19b46f..b1a8352f8b19a 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -66,7 +66,7 @@ type TxnInfo struct { } // ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table -func (info TxnInfo) ToDatum() []types.Datum { +func (info *TxnInfo) ToDatum() []types.Datum { humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) var blockStartTime interface{} if info.BlockStartTime == nil { From 388d0b6d2b757b2e7b1b89b933efea82a6c8ebc1 Mon Sep 17 00:00:00 2001 From: longfangsong Date: Wed, 12 May 2021 13:05:26 +0800 Subject: [PATCH 26/26] move EntriesCount and Size LazyTxn to prevent race --- go.mod | 2 +- go.sum | 2 ++ session/session.go | 1 + session/session_test.go | 8 ++++---- session/txn.go | 30 ++++++++++++++++++++++++++-- session/txninfo/txn_info.go | 4 ++-- store/tikv/unionstore/memdb.go | 23 ++++++++++----------- store/tikv/unionstore/memdb_arena.go | 5 ++--- store/tikv/unionstore/memdb_test.go | 3 +-- 9 files changed, 52 insertions(+), 26 deletions(-) diff --git a/go.mod b/go.mod index bf927f9cc55ce..e4080f4eb61d4 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 // indirect - honnef.co/go/tools v0.1.3 // indirect + honnef.co/go/tools v0.1.4 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 diff --git a/go.sum b/go.sum index 74b4f623789b8..31eaea7c5ac22 100644 --- a/go.sum +++ b/go.sum @@ -934,6 +934,8 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.1.3 h1:qTakTkI6ni6LFD5sBwwsdSO+AQqbSIxOauHTTQKZ/7o= honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.1.4 h1:SadWOkti5uVN1FAMgxn165+Mw00fuQKyk4Gyn/inxNQ= +honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= diff --git a/session/session.go b/session/session.go index 646f109615d80..8fb7e6bff01f3 100644 --- a/session/session.go +++ b/session/session.go @@ -541,6 +541,7 @@ func (s *session) doCommit(ctx context.Context) error { if err = memBuffer.Delete(iter.Key()); err != nil { return errors.Trace(err) } + s.txn.UpdateEntriesCountAndSize() if err = iter.Next(); err != nil { return errors.Trace(err) } diff --git a/session/session_test.go b/session/session_test.go index b54a22660ea2f..7507f49caf5c7 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4336,12 +4336,12 @@ func (s *testTxnStateSuite) TestEntriesCountAndSize(c *C) { tk.MustExec("begin pessimistic;") tk.MustExec("insert into t(a) values (1);") info := tk.Se.TxnInfo() - c.Assert(info.EntriesCount, Equals, int64(1)) - c.Assert(info.EntriesSize, Equals, int64(29)) + c.Assert(info.EntriesCount, Equals, uint64(1)) + c.Assert(info.EntriesSize, Equals, uint64(29)) tk.MustExec("insert into t(a) values (2);") info = tk.Se.TxnInfo() - c.Assert(info.EntriesCount, Equals, int64(2)) - c.Assert(info.EntriesSize, Equals, int64(58)) + c.Assert(info.EntriesCount, Equals, uint64(2)) + c.Assert(info.EntriesSize, Equals, uint64(58)) tk.MustExec("commit;") } diff --git a/session/txn.go b/session/txn.go index 0c1a1dc4d4741..427d61ef92a32 100644 --- a/session/txn.go +++ b/session/txn.go @@ -68,6 +68,10 @@ type LazyTxn struct { State txninfo.TxnRunningState // last trying to block start time blockStartTime unsafe.Pointer // *time.Time, cannot use atomic.Value here because it is possible to be nil + // how many entries are there in the memBuffer, should be equal to self.(kv.Transaction).Len() + EntriesCount uint64 + // how many memory space do the entries in the memBuffer take, should be equal to self.(kv.Transaction).Size() + EntriesSize uint64 } // GetTableInfo returns the cached index name. @@ -83,6 +87,8 @@ func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.EntriesCount, 0) + atomic.StoreUint64(&txn.EntriesSize, 0) } func (txn *LazyTxn) initStmtBuf() { @@ -118,6 +124,8 @@ func (txn *LazyTxn) cleanupStmtBuf() { buf := txn.Transaction.GetMemBuffer() buf.Cleanup(txn.stagingHandle) txn.initCnt = buf.Len() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) } // Size implements the MemBuffer interface. @@ -176,6 +184,8 @@ func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) atomic.StoreUint64(&txn.infoStartTS, kvTxn.StartTS()) txn.initStmtBuf() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) txn.txnFuture = nil } @@ -183,6 +193,8 @@ func (txn *LazyTxn) changeInvalidToPending(future *txnFuture) { txn.Transaction = nil txn.txnFuture = future atomic.StoreUint64(&txn.infoStartTS, 0) + atomic.StoreUint64(&txn.EntriesCount, uint64(0)) + atomic.StoreUint64(&txn.EntriesSize, uint64(0)) } func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { @@ -203,6 +215,8 @@ func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) atomic.StoreUint64(&txn.infoStartTS, t.StartTS()) txn.initStmtBuf() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) return nil } @@ -214,6 +228,8 @@ func (txn *LazyTxn) changeToInvalid() { txn.Transaction = nil txn.txnFuture = nil atomic.StoreUint64(&txn.infoStartTS, 0) + atomic.StoreUint64(&txn.EntriesCount, 0) + atomic.StoreUint64(&txn.EntriesSize, 0) } var hasMockAutoIncIDRetry = int64(0) @@ -366,8 +382,18 @@ func (txn *LazyTxn) Info() *txninfo.TxnInfo { StartTS: startTs, State: atomic.LoadInt32(&txn.State), BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), - EntriesCount: int64(txn.Len()), - EntriesSize: int64(txn.Size()), + EntriesCount: atomic.LoadUint64(&txn.EntriesCount), + EntriesSize: atomic.LoadUint64(&txn.EntriesSize), + } +} + +// UpdateEntriesCountAndSize updates the EntriesCount and EntriesSize +// Note this function is not thread safe, because +// txn.Transaction can be changed during this function's execution. +func (txn *LazyTxn) UpdateEntriesCountAndSize() { + if txn.Valid() { + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) } } diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index b1a8352f8b19a..77a2d8c90cd05 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -51,9 +51,9 @@ type TxnInfo struct { // last trying to block start time BlockStartTime *time.Time // How many entries are in MemDB - EntriesCount int64 + EntriesCount uint64 // MemDB used memory - EntriesSize int64 + EntriesSize uint64 // the following fields will be filled in `session` instead of `LazyTxn` diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index 51db9d2fbf946..7bfc11971e4e2 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -18,7 +18,6 @@ import ( "math" "reflect" "sync" - "sync/atomic" "unsafe" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -60,8 +59,8 @@ type MemDB struct { entrySizeLimit uint64 bufferSizeLimit uint64 - count int64 - size int64 + count int + size int vlogInvalid bool dirty bool @@ -139,8 +138,8 @@ func (db *MemDB) Reset() { db.stages = db.stages[:0] db.dirty = false db.vlogInvalid = false - atomic.StoreInt64(&db.size, 0) - atomic.StoreInt64(&db.count, 0) + db.size = 0 + db.count = 0 db.vlog.reset() db.allocator.reset() } @@ -260,12 +259,12 @@ func (db *MemDB) GetValueByHandle(handle MemKeyHandle) ([]byte, bool) { // Len returns the number of entries in the DB. func (db *MemDB) Len() int { - return int(atomic.LoadInt64(&db.count)) + return db.count } // Size returns sum of keys and values length. func (db *MemDB) Size() int { - return int(atomic.LoadInt64(&db.size)) + return db.size } // Dirty returns whether the root staging buffer is updated. @@ -335,7 +334,7 @@ func (db *MemDB) setValue(x memdbNodeAddr, value []byte) { } } x.vptr = db.vlog.appendValue(x.addr, x.vptr, value) - atomic.AddInt64(&db.size, int64(len(value)-len(oldVal))) + db.size = db.size - len(oldVal) + len(value) } // traverse search for and if not found and insert is true, will add a new node in. @@ -539,8 +538,8 @@ func (db *MemDB) rightRotate(y memdbNodeAddr) { func (db *MemDB) deleteNode(z memdbNodeAddr) { var x, y memdbNodeAddr - atomic.AddInt64(&db.count, -1) - atomic.AddInt64(&db.size, -int64(z.klen)) + db.count-- + db.size -= int(z.klen) if z.left.isNull() || z.right.isNull() { y = z @@ -739,8 +738,8 @@ func (db *MemDB) getRoot() memdbNodeAddr { } func (db *MemDB) allocNode(key []byte) memdbNodeAddr { - atomic.AddInt64(&db.size, int64(len(key))) - atomic.AddInt64(&db.count, int64(1)) + db.size += len(key) + db.count++ x, xn := db.allocator.allocNode(key) return memdbNodeAddr{xn, x} } diff --git a/store/tikv/unionstore/memdb_arena.go b/store/tikv/unionstore/memdb_arena.go index 12818e2282833..45f7f6279e2e5 100644 --- a/store/tikv/unionstore/memdb_arena.go +++ b/store/tikv/unionstore/memdb_arena.go @@ -16,7 +16,6 @@ package unionstore import ( "encoding/binary" "math" - "sync/atomic" "unsafe" "github.com/pingcap/tidb/store/tikv/kv" @@ -310,7 +309,7 @@ func (l *memdbVlog) revertToCheckpoint(db *MemDB, cp *memdbCheckpoint) { node := db.getNode(hdr.nodeAddr) node.vptr = hdr.oldValue - atomic.AddInt64(&db.size, -int64(hdr.valueLen)) + db.size -= int(hdr.valueLen) // oldValue.isNull() == true means this is a newly added value. if hdr.oldValue.isNull() { // If there are no flags associated with this key, we need to delete this node. @@ -322,7 +321,7 @@ func (l *memdbVlog) revertToCheckpoint(db *MemDB, cp *memdbCheckpoint) { db.dirty = true } } else { - atomic.AddInt64(&db.size, int64(len(l.getValue(hdr.oldValue)))) + db.size += len(l.getValue(hdr.oldValue)) } l.moveBackCursor(&cursor, &hdr) diff --git a/store/tikv/unionstore/memdb_test.go b/store/tikv/unionstore/memdb_test.go index 3cac62b70df3c..31097e522fffc 100644 --- a/store/tikv/unionstore/memdb_test.go +++ b/store/tikv/unionstore/memdb_test.go @@ -18,7 +18,6 @@ package unionstore import ( "encoding/binary" "fmt" - "sync/atomic" "testing" . "github.com/pingcap/check" @@ -51,7 +50,7 @@ func (db *MemDB) DeleteKey(key []byte) { if x.isNull() { return } - atomic.AddInt64(&db.size, -int64(len(db.vlog.getValue(x.vptr)))) + db.size -= len(db.vlog.getValue(x.vptr)) db.deleteNode(x) }