Skip to content

Commit

Permalink
renaming
Browse files Browse the repository at this point in the history
  • Loading branch information
MyonKeminta committed May 19, 2021
1 parent 55d8900 commit 97a9d1e
Show file tree
Hide file tree
Showing 9 changed files with 86 additions and 71 deletions.
4 changes: 2 additions & 2 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1535,8 +1535,8 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo
strings.ToLower(infoschema.TableClientErrorsSummaryByHost),
strings.ToLower(infoschema.TableTiDBTrx),
strings.ToLower(infoschema.ClusterTableTiDBTrx),
strings.ToLower(infoschema.TableDeadLock),
strings.ToLower(infoschema.ClusterTableDeadLock):
strings.ToLower(infoschema.TableDeadlocks),
strings.ToLower(infoschema.ClusterTableDeadlocks):
return &MemTableReaderExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()),
table: v.Table,
Expand Down
26 changes: 14 additions & 12 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8159,20 +8159,21 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) {

occurTime := time.Date(2021, 5, 10, 1, 2, 3, 456789000, time.UTC)
rec := &deadlockhistory.DeadlockRecord{
OccurTime: occurTime,
OccurTime: occurTime,
IsRetryable: false,
WaitChain: []deadlockhistory.WaitChainItem{
{
TryLockTxn: 101,
SQLDigest: "aabbccdd",
Key: []byte("k1"),
SQLs: nil,
AllSQLs: nil,
TxnHoldingLock: 102,
},
{
TryLockTxn: 102,
SQLDigest: "ddccbbaa",
Key: []byte("k2"),
SQLs: []string{"sql1"},
AllSQLs: []string{"sql1"},
TxnHoldingLock: 101,
},
},
Expand All @@ -8181,16 +8182,17 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) {

occurTime2 := time.Date(2022, 6, 11, 2, 3, 4, 987654000, time.UTC)
rec2 := &deadlockhistory.DeadlockRecord{
OccurTime: occurTime2,
OccurTime: occurTime2,
IsRetryable: true,
WaitChain: []deadlockhistory.WaitChainItem{
{
TryLockTxn: 201,
SQLs: []string{},
AllSQLs: []string{},
TxnHoldingLock: 202,
},
{
TryLockTxn: 202,
SQLs: []string{"sql1", "sql2, sql3"},
AllSQLs: []string{"sql1", "sql2, sql3"},
TxnHoldingLock: 203,
},
{
Expand All @@ -8207,12 +8209,12 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) {
id2 := strconv.FormatUint(rec2.ID, 10)

tk := testkit.NewTestKit(c, s.store)
tk.MustQuery("select * from information_schema.dead_lock").Check(
tk.MustQuery("select * from information_schema.deadlocks").Check(
testutil.RowsWithSep("/",
id1+"/2021-05-10 01:02:03.456789/101/aabbccdd/6B31/<nil>/102",
id1+"/2021-05-10 01:02:03.456789/102/ddccbbaa/6B32/[sql1]/101",
id2+"/2022-06-11 02:03:04.987654/201/<nil>/<nil>/[]/202",
id2+"/2022-06-11 02:03:04.987654/202/<nil>/<nil>/[sql1, sql2, sql3]/203",
id2+"/2022-06-11 02:03:04.987654/203/<nil>/<nil>/<nil>/201",
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",
))
}
4 changes: 2 additions & 2 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,9 +154,9 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
e.setDataForTiDBTrx(sctx)
case infoschema.ClusterTableTiDBTrx:
err = e.setDataForClusterTiDBTrx(sctx)
case infoschema.TableDeadLock:
case infoschema.TableDeadlocks:
err = e.setDataForDeadlock(sctx)
case infoschema.ClusterTableDeadLock:
case infoschema.ClusterTableDeadlocks:
err = e.setDataForClusterDeadlock(sctx)
}
if err != nil {
Expand Down
6 changes: 3 additions & 3 deletions infoschema/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,8 @@ const (
ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY"
// ClusterTableTiDBTrx is the string constant of cluster transaction running table.
ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX"
// ClusterTableDeadLock is the string constant of cluster dead lock table.
ClusterTableDeadLock = "CLUSTER_DEAD_LOCK"
// ClusterTableDeadlocks is the string constant of cluster dead lock table.
ClusterTableDeadlocks = "CLUSTER_DEADLOCKS"
)

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

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

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

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

var tableDeadLockCols = []columnInfo{
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: "SQLS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"},
{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"},
}

Expand Down Expand Up @@ -1737,7 +1738,7 @@ var tableNameToColumns = map[string][]columnInfo{
TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols,
TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols,
TableTiDBTrx: tableTiDBTrxCols,
TableDeadLock: tableDeadLockCols,
TableDeadlocks: tableDeadlocksCols,
}

func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) {
Expand Down
4 changes: 2 additions & 2 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1539,7 +1539,7 @@ func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) {
Username: "testuser",
Hostname: "localhost",
}, nil, nil), IsTrue)
err := tk.QueryToErr("select * from information_schema.dead_lock")
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")

Expand All @@ -1550,5 +1550,5 @@ func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) {
Username: "testuser2",
Hostname: "localhost",
}, nil, nil), IsTrue)
_ = tk.MustQuery("select * from information_schema.dead_lock")
_ = tk.MustQuery("select * from information_schema.deadlocks")
}
2 changes: 1 addition & 1 deletion session/pessimistic_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -222,7 +222,7 @@ func (s *testPessimisticSuite) TestDeadlock(c *C) {
// 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.dead_lock")
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])
}
Expand Down
39 changes: 21 additions & 18 deletions util/deadlockhistory/deadlock_history.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,17 +32,18 @@ type WaitChainItem struct {
TryLockTxn uint64
SQLDigest string
Key []byte
SQLs []string
AllSQLs []string
TxnHoldingLock uint64
}

// DeadlockRecord represents a deadlock events, and contains multiple transactions' information.
type DeadlockRecord struct {
// The ID doesn't need to be set manually and it's set when it's added into the DeadlockHistory by invoking its Push
// method.
ID uint64
OccurTime time.Time
WaitChain []WaitChainItem
ID uint64
OccurTime time.Time
IsRetryable bool
WaitChain []WaitChainItem
}

// DeadlockHistory is a collection for maintaining recent several deadlock events.
Expand Down Expand Up @@ -118,7 +119,7 @@ func (d *DeadlockHistory) GetAll() []*DeadlockRecord {
}

// GetAllDatum gets all collected deadlock events, and make it into datum that matches the definition of the table
// `INFORMATION_SCHEMA.DEAD_LOCK`.
// `INFORMATION_SCHEMA.DEADLOCKS`.
func (d *DeadlockHistory) GetAllDatum() [][]types.Datum {
records := d.GetAll()
rowsCount := 0
Expand All @@ -128,30 +129,31 @@ func (d *DeadlockHistory) GetAllDatum() [][]types.Datum {

rows := make([][]types.Datum, 0, rowsCount)

row := make([]interface{}, 7)
row := make([]interface{}, 8)
for _, rec := range records {
row[0] = rec.ID
row[1] = types.NewTime(types.FromGoTime(rec.OccurTime), mysql.TypeTimestamp, types.MaxFsp)
row[2] = rec.IsRetryable

for _, item := range rec.WaitChain {
row[2] = item.TryLockTxn
row[3] = item.TryLockTxn

row[3] = nil
row[4] = nil
if len(item.SQLDigest) > 0 {
row[3] = item.SQLDigest
row[4] = item.SQLDigest
}

row[4] = nil
row[5] = nil
if len(item.Key) > 0 {
row[4] = strings.ToUpper(hex.EncodeToString(item.Key))
row[5] = strings.ToUpper(hex.EncodeToString(item.Key))
}

row[5] = nil
if item.SQLs != nil {
row[5] = "[" + strings.Join(item.SQLs, ", ") + "]"
row[6] = nil
if item.AllSQLs != nil {
row[6] = "[" + strings.Join(item.AllSQLs, ", ") + "]"
}

row[6] = item.TxnHoldingLock
row[7] = item.TxnHoldingLock

rows = append(rows, types.MakeDatums(row...))
}
Expand Down Expand Up @@ -183,13 +185,14 @@ func ErrDeadlockToDeadlockRecord(dl *tikverr.ErrDeadlock) *DeadlockRecord {
TryLockTxn: rawItem.Txn,
SQLDigest: sqlDigest,
Key: rawItem.Key,
SQLs: nil,
AllSQLs: nil,
TxnHoldingLock: rawItem.WaitForTxn,
})
}
rec := &DeadlockRecord{
OccurTime: time.Now(),
WaitChain: waitChain,
OccurTime: time.Now(),
IsRetryable: dl.IsRetryable,
WaitChain: waitChain,
}
return rec
}
57 changes: 33 additions & 24 deletions util/deadlockhistory/deadlock_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,13 +140,14 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) {

h := NewDeadlockHistory(10)
h.Push(&DeadlockRecord{
OccurTime: time1,
OccurTime: time1,
IsRetryable: false,
WaitChain: []WaitChainItem{
{
TryLockTxn: 101,
SQLDigest: "sql1",
Key: []byte("k1"),
SQLs: []string{"sql1", "sql2"},
AllSQLs: []string{"sql1", "sql2"},
TxnHoldingLock: 102,
},
// It should work even some information are missing.
Expand All @@ -157,30 +158,32 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) {
},
})
h.Push(&DeadlockRecord{
OccurTime: time2,
OccurTime: time2,
IsRetryable: true,
WaitChain: []WaitChainItem{
{
TryLockTxn: 201,
SQLs: []string{},
AllSQLs: []string{},
TxnHoldingLock: 202,
},
{
TryLockTxn: 202,
SQLs: []string{"sql1"},
AllSQLs: []string{"sql1"},
TxnHoldingLock: 201,
},
},
})
// A deadlock error without wait chain shows nothing in the query result.
h.Push(&DeadlockRecord{
OccurTime: time.Now(),
WaitChain: nil,
OccurTime: time.Now(),
IsRetryable: false,
WaitChain: nil,
})

res := h.GetAllDatum()
c.Assert(len(res), Equals, 4)
for _, row := range res {
c.Assert(len(row), Equals, 7)
c.Assert(len(row), Equals, 8)
}

toGoTime := func(d types.Datum) time.Time {
Expand All @@ -193,31 +196,35 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) {

c.Assert(res[0][0].GetValue(), Equals, uint64(1)) // ID
c.Assert(toGoTime(res[0][1]), Equals, time1) // OCCUR_TIME
c.Assert(res[0][2].GetValue(), Equals, uint64(101)) // TRY_LOCK_TRX_ID
c.Assert(res[0][3].GetValue(), Equals, "sql1") // SQL_DIGEST
c.Assert(res[0][4].GetValue(), Equals, "6B31") // KEY
c.Assert(res[0][5].GetValue(), Equals, "[sql1, sql2]") // SQLS
c.Assert(res[0][6].GetValue(), Equals, uint64(102)) // TRX_HOLDING_LOCK
c.Assert(res[0][2].GetValue(), Equals, int64(0)) // RETRYABLE
c.Assert(res[0][3].GetValue(), Equals, uint64(101)) // TRY_LOCK_TRX_ID
c.Assert(res[0][4].GetValue(), Equals, "sql1") // SQL_DIGEST
c.Assert(res[0][5].GetValue(), Equals, "6B31") // KEY
c.Assert(res[0][6].GetValue(), Equals, "[sql1, sql2]") // ALL_SQLS
c.Assert(res[0][7].GetValue(), Equals, uint64(102)) // TRX_HOLDING_LOCK

c.Assert(res[1][0].GetValue(), Equals, uint64(1)) // ID
c.Assert(toGoTime(res[1][1]), Equals, time1) // OCCUR_TIME
c.Assert(res[1][2].GetValue(), Equals, uint64(102)) // TRY_LOCK_TRX_ID
c.Assert(res[1][3].GetValue(), Equals, nil) // SQL_DIGEST
c.Assert(res[1][4].GetValue(), Equals, nil) // KEY
c.Assert(res[1][5].GetValue(), Equals, nil) // SQLS
c.Assert(res[1][6].GetValue(), Equals, uint64(101)) // TRX_HOLDING_LOCK
c.Assert(res[1][2].GetValue(), Equals, int64(0)) // RETRYABLE
c.Assert(res[1][3].GetValue(), Equals, uint64(102)) // TRY_LOCK_TRX_ID
c.Assert(res[1][4].GetValue(), Equals, nil) // SQL_DIGEST
c.Assert(res[1][5].GetValue(), Equals, nil) // KEY
c.Assert(res[1][6].GetValue(), Equals, nil) // ALL_SQLS
c.Assert(res[1][7].GetValue(), Equals, uint64(101)) // TRX_HOLDING_LOCK

c.Assert(res[2][0].GetValue(), Equals, uint64(2)) // ID
c.Assert(toGoTime(res[2][1]), Equals, time2) // OCCUR_TIME
c.Assert(res[2][2].GetValue(), Equals, uint64(201)) // TRY_LOCK_TRX_ID
c.Assert(res[2][5].GetValue(), Equals, "[]") // SQLS
c.Assert(res[2][6].GetValue(), Equals, uint64(202)) // TRX_HOLDING_LOCK
c.Assert(res[2][2].GetValue(), Equals, int64(1)) // RETRYABLE
c.Assert(res[2][3].GetValue(), Equals, uint64(201)) // TRY_LOCK_TRX_ID
c.Assert(res[2][6].GetValue(), Equals, "[]") // ALL_SQLS
c.Assert(res[2][7].GetValue(), Equals, uint64(202)) // TRX_HOLDING_LOCK

c.Assert(res[3][0].GetValue(), Equals, uint64(2)) // ID
c.Assert(toGoTime(res[3][1]), Equals, time2) // OCCUR_TIME
c.Assert(res[3][2].GetValue(), Equals, uint64(202)) // TRY_LOCK_TRX_ID
c.Assert(res[3][5].GetValue(), Equals, "[sql1]") // SQLS
c.Assert(res[3][6].GetValue(), Equals, uint64(201)) // TRX_HOLDING_LOCK
c.Assert(res[3][2].GetValue(), Equals, int64(1)) // RETRYABLE
c.Assert(res[3][3].GetValue(), Equals, uint64(202)) // TRY_LOCK_TRX_ID
c.Assert(res[3][6].GetValue(), Equals, "[sql1]") // ALL_SQLS
c.Assert(res[3][7].GetValue(), Equals, uint64(201)) // TRX_HOLDING_LOCK
}

func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) {
Expand All @@ -241,9 +248,11 @@ func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) {
},
},
},
IsRetryable: true,
}

expectedRecord := &DeadlockRecord{
IsRetryable: true,
WaitChain: []WaitChainItem{
{
TryLockTxn: 100,
Expand Down

0 comments on commit 97a9d1e

Please sign in to comment.