Skip to content

Commit

Permalink
infoschema, executor: Add the deadlock table (#24524)
Browse files Browse the repository at this point in the history
  • Loading branch information
MyonKeminta authored May 19, 2021
1 parent 15dfd7b commit 3904ca3
Show file tree
Hide file tree
Showing 12 changed files with 668 additions and 14 deletions.
4 changes: 3 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1534,7 +1534,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo
strings.ToLower(infoschema.TableClientErrorsSummaryByUser),
strings.ToLower(infoschema.TableClientErrorsSummaryByHost),
strings.ToLower(infoschema.TableTiDBTrx),
strings.ToLower(infoschema.ClusterTableTiDBTrx):
strings.ToLower(infoschema.ClusterTableTiDBTrx),
strings.ToLower(infoschema.TableDeadlocks),
strings.ToLower(infoschema.ClusterTableDeadlocks):
return &MemTableReaderExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()),
table: v.Table,
Expand Down
9 changes: 9 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/tikv"
tikverr "github.com/pingcap/tidb/store/tikv/error"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
tikvutil "github.com/pingcap/tidb/store/tikv/util"
"github.com/pingcap/tidb/table"
Expand All @@ -58,6 +59,7 @@ import (
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/deadlockhistory"
"github.com/pingcap/tidb/util/disk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -983,6 +985,13 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc
LockKeysCount: &seVars.StmtCtx.LockKeysCount,
LockExpired: &seVars.TxnCtx.LockExpire,
ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest),
OnDeadlock: func(deadlock *tikverr.ErrDeadlock) {
// TODO: Support collecting retryable deadlocks according to the config.
if !deadlock.IsRetryable {
rec := deadlockhistory.ErrDeadlockToDeadlockRecord(deadlock)
deadlockhistory.GlobalDeadlockHistory.Push(rec)
}
},
}
}

Expand Down
65 changes: 65 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/deadlockhistory"
"github.com/pingcap/tidb/util/gcutil"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
Expand Down Expand Up @@ -8151,7 +8152,71 @@ func (s *testSerialSuite) TestIssue24210(c *C) {
c.Assert(err.Error(), Equals, "mock SelectionExec.baseExecutor.Open returned error")
err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError")
c.Assert(err, IsNil)
}

func (s *testSerialSuite) TestDeadlockTable(c *C) {
deadlockhistory.GlobalDeadlockHistory.Clear()

occurTime := time.Date(2021, 5, 10, 1, 2, 3, 456789000, time.UTC)
rec := &deadlockhistory.DeadlockRecord{
OccurTime: occurTime,
IsRetryable: false,
WaitChain: []deadlockhistory.WaitChainItem{
{
TryLockTxn: 101,
SQLDigest: "aabbccdd",
Key: []byte("k1"),
AllSQLs: nil,
TxnHoldingLock: 102,
},
{
TryLockTxn: 102,
SQLDigest: "ddccbbaa",
Key: []byte("k2"),
AllSQLs: []string{"sql1"},
TxnHoldingLock: 101,
},
},
}
deadlockhistory.GlobalDeadlockHistory.Push(rec)

occurTime2 := time.Date(2022, 6, 11, 2, 3, 4, 987654000, time.UTC)
rec2 := &deadlockhistory.DeadlockRecord{
OccurTime: occurTime2,
IsRetryable: true,
WaitChain: []deadlockhistory.WaitChainItem{
{
TryLockTxn: 201,
AllSQLs: []string{},
TxnHoldingLock: 202,
},
{
TryLockTxn: 202,
AllSQLs: []string{"sql1", "sql2, sql3"},
TxnHoldingLock: 203,
},
{
TryLockTxn: 203,
TxnHoldingLock: 201,
},
},
}
deadlockhistory.GlobalDeadlockHistory.Push(rec2)

// `Push` sets the record's ID, and ID in a single DeadlockHistory is monotonically increasing. We must get it here
// to know what it is.
id1 := strconv.FormatUint(rec.ID, 10)
id2 := strconv.FormatUint(rec2.ID, 10)

tk := testkit.NewTestKit(c, s.store)
tk.MustQuery("select * from information_schema.deadlocks").Check(
testutil.RowsWithSep("/",
id1+"/2021-05-10 01:02:03.456789/0/101/aabbccdd/6B31/<nil>/102",
id1+"/2021-05-10 01:02:03.456789/0/102/ddccbbaa/6B32/[sql1]/101",
id2+"/2022-06-11 02:03:04.987654/1/201/<nil>/<nil>/[]/202",
id2+"/2022-06-11 02:03:04.987654/1/202/<nil>/<nil>/[sql1, sql2, sql3]/203",
id2+"/2022-06-11 02:03:04.987654/1/203/<nil>/<nil>/<nil>/201",
))
}

func (s testSerialSuite) TestExprBlackListForEnum(c *C) {
Expand Down
32 changes: 32 additions & 0 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/deadlockhistory"
"github.com/pingcap/tidb/util/pdapi"
"github.com/pingcap/tidb/util/sem"
"github.com/pingcap/tidb/util/set"
Expand Down Expand Up @@ -153,6 +154,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
e.setDataForTiDBTrx(sctx)
case infoschema.ClusterTableTiDBTrx:
err = e.setDataForClusterTiDBTrx(sctx)
case infoschema.TableDeadlocks:
err = e.setDataForDeadlock(sctx)
case infoschema.ClusterTableDeadlocks:
err = e.setDataForClusterDeadlock(sctx)
}
if err != nil {
return nil, err
Expand Down Expand Up @@ -2048,6 +2053,33 @@ func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) err
return nil
}

func (e *memtableRetriever) setDataForDeadlock(ctx sessionctx.Context) error {
hasPriv := false
if pm := privilege.GetPrivilegeManager(ctx); pm != nil {
hasPriv = pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv)
}

if !hasPriv {
return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS")
}

e.rows = deadlockhistory.GlobalDeadlockHistory.GetAllDatum()
return nil
}

func (e *memtableRetriever) setDataForClusterDeadlock(ctx sessionctx.Context) error {
err := e.setDataForDeadlock(ctx)
if err != nil {
return err
}
rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows)
if err != nil {
return err
}
e.rows = rows
return nil
}

type hugeMemTableRetriever struct {
dummyCloser
table *model.TableInfo
Expand Down
3 changes: 3 additions & 0 deletions infoschema/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@ const (
ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY"
// ClusterTableTiDBTrx is the string constant of cluster transaction running table.
ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX"
// ClusterTableDeadlocks is the string constant of cluster dead lock table.
ClusterTableDeadlocks = "CLUSTER_DEADLOCKS"
)

// memTableToClusterTables means add memory table to cluster table.
Expand All @@ -48,6 +50,7 @@ var memTableToClusterTables = map[string]string{
TableStatementsSummary: ClusterTableStatementsSummary,
TableStatementsSummaryHistory: ClusterTableStatementsSummaryHistory,
TableTiDBTrx: ClusterTableTiDBTrx,
TableDeadlocks: ClusterTableDeadlocks,
}

func init() {
Expand Down
16 changes: 16 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,8 @@ const (
TableClientErrorsSummaryByHost = "CLIENT_ERRORS_SUMMARY_BY_HOST"
// TableTiDBTrx is current running transaction status table.
TableTiDBTrx = "TIDB_TRX"
// TableDeadlocks is the string constatnt of deadlock table.
TableDeadlocks = "DEADLOCKS"
)

var tableIDMap = map[string]int64{
Expand Down Expand Up @@ -239,6 +241,8 @@ var tableIDMap = map[string]int64{
TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69,
TableTiDBTrx: autoid.InformationSchemaDBID + 70,
ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 71,
TableDeadlocks: autoid.InformationSchemaDBID + 72,
ClusterTableDeadlocks: autoid.InformationSchemaDBID + 73,
}

type columnInfo struct {
Expand Down Expand Up @@ -1353,6 +1357,17 @@ var tableTiDBTrxCols = []columnInfo{
{name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"},
}

var tableDeadlocksCols = []columnInfo{
{name: "DEADLOCK_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The ID to dinstinguish different deadlock events"},
{name: "OCCUR_TIME", tp: mysql.TypeTimestamp, decimal: 6, size: 26, comment: "The physical time when the deadlock occurs"},
{name: "RETRYABLE", tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether the deadlock is retryable. Retryable deadlocks are usually not reported to the client"},
{name: "TRY_LOCK_TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's trying to acquire the lock"},
{name: "CURRENT_SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "The digest of the SQL that's being blocked"},
{name: "KEY", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The key on which a transaction is waiting for another"},
{name: "ALL_SQLS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"},
{name: "TRX_HOLDING_LOCK", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's currently holding the lock"},
}

// 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.
Expand Down Expand Up @@ -1723,6 +1738,7 @@ var tableNameToColumns = map[string][]columnInfo{
TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols,
TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols,
TableTiDBTrx: tableTiDBTrxCols,
TableDeadlocks: tableDeadlocksCols,
}

func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) {
Expand Down
21 changes: 21 additions & 0 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1531,3 +1531,24 @@ func (s *testTableSuite) TestTrx(c *C) {
testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest + " Normal <nil> 1 19 2 root test"),
)
}

func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) {
tk := s.newTestKitWithRoot(c)
tk.MustExec("create user 'testuser'@'localhost'")
c.Assert(tk.Se.Auth(&auth.UserIdentity{
Username: "testuser",
Hostname: "localhost",
}, nil, nil), IsTrue)
err := tk.QueryToErr("select * from information_schema.deadlocks")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation")

tk = s.newTestKitWithRoot(c)
tk.MustExec("create user 'testuser2'@'localhost'")
tk.MustExec("grant process on *.* to 'testuser2'@'localhost'")
c.Assert(tk.Se.Auth(&auth.UserIdentity{
Username: "testuser2",
Hostname: "localhost",
}, nil, nil), IsTrue)
_ = tk.MustQuery("select * from information_schema.deadlocks")
}
48 changes: 36 additions & 12 deletions session/pessimistic_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package session_test
import (
"context"
"fmt"
"strconv"
"strings"
"sync"
"sync/atomic"
Expand All @@ -24,6 +25,7 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
Expand All @@ -37,6 +39,7 @@ import (
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/deadlockhistory"
"github.com/pingcap/tidb/util/testkit"
)

Expand Down Expand Up @@ -171,27 +174,33 @@ func (s *testPessimisticSuite) TestTxnMode(c *C) {
}

func (s *testPessimisticSuite) TestDeadlock(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists deadlock")
tk.MustExec("create table deadlock (k int primary key, v int)")
tk.MustExec("insert into deadlock values (1, 1), (2, 1)")
deadlockhistory.GlobalDeadlockHistory.Clear()

tk1 := testkit.NewTestKitWithInit(c, s.store)
tk1.MustExec("drop table if exists deadlock")
tk1.MustExec("create table deadlock (k int primary key, v int)")
tk1.MustExec("insert into deadlock values (1, 1), (2, 1)")
tk1.MustExec("begin pessimistic")
tk1.MustExec("update deadlock set v = v + 1 where k = 1")
ts1, err := strconv.ParseUint(tk1.MustQuery("select @@tidb_current_ts").Rows()[0][0].(string), 10, 64)
c.Assert(err, IsNil)

tk2 := testkit.NewTestKitWithInit(c, s.store)
tk2.MustExec("begin pessimistic")
ts2, err := strconv.ParseUint(tk2.MustQuery("select @@tidb_current_ts").Rows()[0][0].(string), 10, 64)
c.Assert(err, IsNil)

syncCh := make(chan error)
go func() {
tk1 := testkit.NewTestKitWithInit(c, s.store)
tk1.MustExec("begin pessimistic")
tk1.MustExec("update deadlock set v = v + 1 where k = 2")
tk2.MustExec("update deadlock set v = v + 1 where k = 2")
syncCh <- nil
_, err := tk1.Exec("update deadlock set v = v + 1 where k = 1")
_, err := tk2.Exec("update deadlock set v = v + 1 where k = 1")
syncCh <- err
}()
tk.MustExec("begin pessimistic")
tk.MustExec("update deadlock set v = v + 1 where k = 1")
<-syncCh
_, err1 := tk.Exec("update deadlock set v = v + 1 where k = 2")
_, err1 := tk1.Exec("update deadlock set v = v + 1 where k = 2")
err2 := <-syncCh
// Either err1 or err2 is deadlock error.
var err error
if err1 != nil {
c.Assert(err2, IsNil)
err = err1
Expand All @@ -201,6 +210,21 @@ func (s *testPessimisticSuite) TestDeadlock(c *C) {
e, ok := errors.Cause(err).(*terror.Error)
c.Assert(ok, IsTrue)
c.Assert(int(e.Code()), Equals, mysql.ErrLockDeadlock)

_, digest := parser.NormalizeDigest("update deadlock set v = v + 1 where k = 1")

expectedDeadlockInfo := []string{
fmt.Sprintf("%v %v %v", ts1, ts2, digest),
fmt.Sprintf("%v %v %v", ts2, ts1, digest),
}
// The last one is the transaction that encountered the deadlock error.
if err1 != nil {
// Swap the two to match the correct order.
expectedDeadlockInfo[0], expectedDeadlockInfo[1] = expectedDeadlockInfo[1], expectedDeadlockInfo[0]
}
res := tk1.MustQuery("select deadlock_id, try_lock_trx_id, trx_holding_lock, current_sql_digest from information_schema.deadlocks")
res.CheckAt([]int{1, 2, 3}, testkit.Rows(expectedDeadlockInfo...))
c.Assert(res.Rows()[0][0], Equals, res.Rows()[1][0])
}

func (s *testPessimisticSuite) TestSingleStatementRollback(c *C) {
Expand Down
2 changes: 2 additions & 0 deletions store/tikv/kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import (
"sync"
"time"

tikverr "github.com/pingcap/tidb/store/tikv/error"
"github.com/pingcap/tidb/store/tikv/util"
)

Expand All @@ -28,4 +29,5 @@ type LockCtx struct {
LockExpired *uint32
Stats *util.LockKeysDetails
ResourceGroupTag []byte
OnDeadlock func(*tikverr.ErrDeadlock)
}
7 changes: 6 additions & 1 deletion store/tikv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -642,8 +642,13 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput
keyMayBeLocked := !(tikverr.IsErrWriteConflict(err) || tikverr.IsErrKeyExist(err))
// If there is only 1 key and lock fails, no need to do pessimistic rollback.
if len(keys) > 1 || keyMayBeLocked {
dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock)
if ok && lockCtx.OnDeadlock != nil {
// Call OnDeadlock before pessimistic rollback.
lockCtx.OnDeadlock(dl)
}
wg := txn.asyncPessimisticRollback(ctx, keys)
if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok {
if ok {
logutil.Logger(ctx).Debug("deadlock error received", zap.Uint64("startTS", txn.startTS), zap.Stringer("deadlockInfo", dl))
if hashInKeys(dl.DeadlockKeyHash, keys) {
dl.IsRetryable = true
Expand Down
Loading

0 comments on commit 3904ca3

Please sign in to comment.