Skip to content

Commit

Permalink
txn: Slow txn log (#41864)
Browse files Browse the repository at this point in the history
close #41863
  • Loading branch information
ekexium authored Mar 6, 2023
1 parent 196ddc0 commit 9e27518
Show file tree
Hide file tree
Showing 9 changed files with 93 additions and 2 deletions.
1 change: 1 addition & 0 deletions session/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,7 @@ go_library(
"@com_github_tikv_client_go_v2//util",
"@io_etcd_go_etcd_client_v3//concurrency",
"@org_uber_go_zap//:zap",
"@org_uber_go_zap//zapcore",
],
)

Expand Down
1 change: 1 addition & 0 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2120,6 +2120,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex
cmd32 := atomic.LoadUint32(&s.GetSessionVars().CommandValue)
s.SetProcessInfo(stmtNode.Text(), time.Now(), byte(cmd32), 0)
s.txn.onStmtStart(digest.String())
defer sessiontxn.GetTxnManager(s).OnStmtEnd()
defer s.txn.onStmtEnd()

if err := s.onTxnManagerStmtStartOrRetry(ctx, stmtNode); err != nil {
Expand Down
72 changes: 72 additions & 0 deletions session/txnmanager.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,21 @@ package session
import (
"context"
"fmt"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/sessiontxn/isolation"
"github.com/pingcap/tidb/sessiontxn/staleread"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
)

func init() {
Expand All @@ -52,6 +57,22 @@ type txnManager struct {

// We always reuse the same OptimisticTxnContextProvider in one session to reduce memory allocation cost for every new txn.
reservedOptimisticProviders [2]isolation.OptimisticTxnContextProvider

// used for slow transaction logs
events []event
lastInstant time.Time
enterTxnInstant time.Time
}

type event struct {
event string
duration time.Duration
}

func (s event) MarshalLogObject(enc zapcore.ObjectEncoder) error {
enc.AddString("event", s.event)
enc.AddDuration("gap", s.duration)
return nil
}

func newTxnManager(sctx sessionctx.Context) *txnManager {
Expand Down Expand Up @@ -153,12 +174,30 @@ func (m *txnManager) EnterNewTxn(ctx context.Context, r *sessiontxn.EnterNewTxnR
if r.Type == sessiontxn.EnterNewTxnWithBeginStmt {
m.sctx.GetSessionVars().SetInTxn(true)
}

m.resetEvents()
m.recordEvent("enter txn")
return nil
}

func (m *txnManager) OnTxnEnd() {
m.ctxProvider = nil
m.stmtNode = nil

m.events = append(m.events, event{event: "txn end", duration: time.Since(m.lastInstant)})

duration := time.Since(m.enterTxnInstant)
threshold := m.sctx.GetSessionVars().SlowTxnThreshold
if threshold > 0 && uint64(duration.Milliseconds()) >= threshold {
logutil.BgLogger().Info(
"slow transaction", zap.Duration("duration", duration),
zap.Uint64("conn", m.sctx.GetSessionVars().ConnectionID),
zap.Uint64("txnStartTS", m.sctx.GetSessionVars().TxnCtx.StartTS),
zap.Objects("events", m.events),
)
}

m.lastInstant = time.Now()
}

func (m *txnManager) GetCurrentStmt() ast.StmtNode {
Expand All @@ -172,9 +211,23 @@ func (m *txnManager) OnStmtStart(ctx context.Context, node ast.StmtNode) error {
if m.ctxProvider == nil {
return errors.New("context provider not set")
}

var sql string
if node != nil {
sql = node.OriginalText()
if m.sctx.GetSessionVars().EnableRedactLog {
sql = parser.Normalize(sql)
}
}
m.recordEvent(sql)
return m.ctxProvider.OnStmtStart(ctx, m.stmtNode)
}

// OnStmtEnd implements the TxnManager interface
func (m *txnManager) OnStmtEnd() {
m.recordEvent("stmt end")
}

// OnPessimisticStmtStart is the hook that should be called when starts handling a pessimistic DML or
// a pessimistic select-for-update statements.
func (m *txnManager) OnPessimisticStmtStart(ctx context.Context) error {
Expand Down Expand Up @@ -222,14 +275,33 @@ func (m *txnManager) OnStmtCommit(ctx context.Context) error {
if m.ctxProvider == nil {
return errors.New("context provider not set")
}
m.recordEvent("stmt commit")
return m.ctxProvider.OnStmtCommit(ctx)
}

func (m *txnManager) recordEvent(eventName string) {
if m.events == nil {
m.resetEvents()
}
m.events = append(m.events, event{event: eventName, duration: time.Since(m.lastInstant)})
m.lastInstant = time.Now()
}

func (m *txnManager) resetEvents() {
if m.events == nil {
m.events = make([]event, 0, 10)
} else {
m.events = m.events[:0]
}
m.enterTxnInstant = time.Now()
}

// OnStmtRollback is the hook that should be called when a statement fails to execute.
func (m *txnManager) OnStmtRollback(ctx context.Context, isForPessimisticRetry bool) error {
if m.ctxProvider == nil {
return errors.New("context provider not set")
}
m.recordEvent("stmt rollback")
return m.ctxProvider.OnStmtRollback(ctx, isForPessimisticRetry)
}

Expand Down
3 changes: 3 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1374,6 +1374,9 @@ type SessionVars struct {
// For now it is not public to user
EnableINLJoinInnerMultiPattern bool

// SlowTxnThreshold is the threshold of slow transaction logs
SlowTxnThreshold uint64

// LoadBasedReplicaReadThreshold is the threshold for the estimated wait duration of a store.
// If exceeding the threshold, try other stores using replica read.
LoadBasedReplicaReadThreshold time.Duration
Expand Down
6 changes: 6 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,12 @@ var defaultSysVars = []*SysVar{
}, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) {
return BoolToOnOff(ProcessGeneralLog.Load()), nil
}},
{Scope: ScopeSession, Name: TiDBSlowTxnLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowTxnThreshold),
Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error {
s.SlowTxnThreshold = TidbOptUint64(val, logutil.DefaultSlowTxnThreshold)
return nil
},
},
{Scope: ScopeInstance, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
atomic.StoreUint64(&config.GetGlobalConfig().Instance.SlowThreshold, uint64(TidbOptInt64(val, logutil.DefaultSlowThreshold)))
return nil
Expand Down
3 changes: 3 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,9 @@ const (
// TiDBSlowLogThreshold is used to set the slow log threshold in the server.
TiDBSlowLogThreshold = "tidb_slow_log_threshold"

// TiDBSlowTxnLogThreshold is used to set the slow transaction log threshold in the server.
TiDBSlowTxnLogThreshold = "tidb_slow_txn_log_threshold"

// TiDBRecordPlanInSlowLog is used to log the plan of the slow query.
TiDBRecordPlanInSlowLog = "tidb_record_plan_in_slow_log"

Expand Down
6 changes: 4 additions & 2 deletions sessiontxn/interface.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ type EnterNewTxnRequest struct {
// causalConsistencyOnly means whether enable causal consistency for transactions, default is false
CausalConsistencyOnly bool
// staleReadTS indicates the read ts for the stale read transaction.
//The default value is zero which means not a stale read transaction.
// The default value is zero which means not a stale read transaction.
StaleReadTS uint64
}

Expand Down Expand Up @@ -123,7 +123,7 @@ type TxnContextProvider interface {
GetTxnScope() string
// GetReadReplicaScope returns the read replica scope
GetReadReplicaScope() string
//GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update)
// GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update)
GetStmtReadTS() (uint64, error)
// GetStmtForUpdateTS returns the read timestamp used by update/insert/delete or select ... for update
GetStmtForUpdateTS() (uint64, error)
Expand Down Expand Up @@ -205,6 +205,8 @@ type TxnManager interface {
OnStmtCommit(ctx context.Context) error
// OnStmtRollback is the hook that should be called when a statement fails to execute.
OnStmtRollback(ctx context.Context, isForPessimisticRetry bool) error
// OnStmtEnd is called when a statement ends, together with txn.onStmtEnd()
OnStmtEnd()
// OnLocalTemporaryTableCreated is the hook that should be called when a local temporary table created.
OnLocalTemporaryTableCreated()
// ActivateTxn activates the transaction.
Expand Down
2 changes: 2 additions & 0 deletions util/logutil/log.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ const (
DefaultLogFormat = "text"
// DefaultSlowThreshold is the default slow log threshold in millisecond.
DefaultSlowThreshold = 300
// DefaultSlowTxnThreshold is the default slow txn log threshold in ms.
DefaultSlowTxnThreshold = 0
// DefaultQueryLogMaxLen is the default max length of the query in the log.
DefaultQueryLogMaxLen = 4096
// DefaultRecordPlanInSlowLog is the default value for whether enable log query plan in the slow log.
Expand Down
1 change: 1 addition & 0 deletions util/sem/sem.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,7 @@ func IsInvisibleSysVar(varNameInLower string) bool {
variable.TiDBRowFormatVersion,
variable.TiDBSlowQueryFile,
variable.TiDBSlowLogThreshold,
variable.TiDBSlowTxnLogThreshold,
variable.TiDBEnableCollectExecutionInfo,
variable.TiDBMemoryUsageAlarmRatio,
variable.TiDBRedactLog,
Expand Down

0 comments on commit 9e27518

Please sign in to comment.