Skip to content

Commit

Permalink
*: Add system variable tidb_session_alias to log a custom field `se…
Browse files Browse the repository at this point in the history
…ssion_alias` in session log (pingcap#46072)

close pingcap#46071
  • Loading branch information
lcwangchao committed Aug 18, 2023
1 parent 883e88a commit ddb7b36
Show file tree
Hide file tree
Showing 18 changed files with 251 additions and 55 deletions.
8 changes: 8 additions & 0 deletions parser/model/model.go
Original file line number Diff line number Diff line change
Expand Up @@ -2167,3 +2167,11 @@ func (s WindowRepeatType) String() string {
return ""
}
}

// TraceInfo is the information for trace.
type TraceInfo struct {
// ConnectionID is the id of the connection
ConnectionID uint64 `json:"connection_id"`
// SessionAlias is the alias of session
SessionAlias string `json:"session_alias"`
}
52 changes: 20 additions & 32 deletions planner/core/preprocess.go
Original file line number Diff line number Diff line change
Expand Up @@ -785,32 +785,32 @@ func (p *preprocessor) checkSetOprSelectList(stmt *ast.SetOprSelectList) {
}

func (p *preprocessor) checkCreateDatabaseGrammar(stmt *ast.CreateDatabaseStmt) {
if isIncorrectName(stmt.Name.L) {
if util.IsInCorrectIdentifierName(stmt.Name.L) {
p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name)
}
}

func (p *preprocessor) checkAlterDatabaseGrammar(stmt *ast.AlterDatabaseStmt) {
// for 'ALTER DATABASE' statement, database name can be empty to alter default database.
if isIncorrectName(stmt.Name.L) && !stmt.AlterDefaultDatabase {
if util.IsInCorrectIdentifierName(stmt.Name.L) && !stmt.AlterDefaultDatabase {
p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name)
}
}

func (p *preprocessor) checkDropDatabaseGrammar(stmt *ast.DropDatabaseStmt) {
if isIncorrectName(stmt.Name.L) {
if util.IsInCorrectIdentifierName(stmt.Name.L) {
p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.Name)
}
}

func (p *preprocessor) checkFlashbackTableGrammar(stmt *ast.FlashBackTableStmt) {
if isIncorrectName(stmt.NewName) {
if util.IsInCorrectIdentifierName(stmt.NewName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(stmt.NewName)
}
}

func (p *preprocessor) checkFlashbackDatabaseGrammar(stmt *ast.FlashBackDatabaseStmt) {
if isIncorrectName(stmt.NewName) {
if util.IsInCorrectIdentifierName(stmt.NewName) {
p.err = dbterror.ErrWrongDBName.GenWithStackByArgs(stmt.NewName)
}
}
Expand Down Expand Up @@ -874,7 +874,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) {
}
}
tName := stmt.Table.Name.String()
if isIncorrectName(tName) {
if util.IsInCorrectIdentifierName(tName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(tName)
return
}
Expand Down Expand Up @@ -938,7 +938,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) {
if stmt.Partition != nil {
for _, def := range stmt.Partition.Definitions {
pName := def.Name.String()
if isIncorrectName(pName) {
if util.IsInCorrectIdentifierName(pName) {
p.err = dbterror.ErrWrongPartitionName.GenWithStackByArgs()
return
}
Expand All @@ -948,12 +948,12 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) {

func (p *preprocessor) checkCreateViewGrammar(stmt *ast.CreateViewStmt) {
vName := stmt.ViewName.Name.String()
if isIncorrectName(vName) {
if util.IsInCorrectIdentifierName(vName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(vName)
return
}
for _, col := range stmt.Cols {
if isIncorrectName(col.String()) {
if util.IsInCorrectIdentifierName(col.String()) {
p.err = dbterror.ErrWrongColumnName.GenWithStackByArgs(col)
return
}
Expand Down Expand Up @@ -1014,7 +1014,7 @@ func (p *preprocessor) checkDropTableGrammar(stmt *ast.DropTableStmt) {
func (p *preprocessor) checkDropTemporaryTableGrammar(stmt *ast.DropTableStmt) {
currentDB := model.NewCIStr(p.sctx.GetSessionVars().CurrentDB)
for _, t := range stmt.Tables {
if isIncorrectName(t.Name.String()) {
if util.IsInCorrectIdentifierName(t.Name.String()) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(t.Name.String())
return
}
Expand Down Expand Up @@ -1045,7 +1045,7 @@ func (p *preprocessor) checkDropTemporaryTableGrammar(stmt *ast.DropTableStmt) {

func (p *preprocessor) checkDropTableNames(tables []*ast.TableName) {
for _, t := range tables {
if isIncorrectName(t.Name.String()) {
if util.IsInCorrectIdentifierName(t.Name.String()) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(t.Name.String())
return
}
Expand Down Expand Up @@ -1118,7 +1118,7 @@ func checkColumnOptions(isTempTable bool, ops []*ast.ColumnOption) (int, error)

func (p *preprocessor) checkCreateIndexGrammar(stmt *ast.CreateIndexStmt) {
tName := stmt.Table.Name.String()
if isIncorrectName(tName) {
if util.IsInCorrectIdentifierName(tName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(tName)
return
}
Expand Down Expand Up @@ -1152,12 +1152,12 @@ func (p *preprocessor) checkRenameTableGrammar(stmt *ast.RenameTableStmt) {
}

func (p *preprocessor) checkRenameTable(oldTable, newTable string) {
if isIncorrectName(oldTable) {
if util.IsInCorrectIdentifierName(oldTable) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(oldTable)
return
}

if isIncorrectName(newTable) {
if util.IsInCorrectIdentifierName(newTable) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(newTable)
return
}
Expand All @@ -1182,15 +1182,15 @@ func (p *preprocessor) checkRepairTableGrammar(stmt *ast.RepairTableStmt) {

func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) {
tName := stmt.Table.Name.String()
if isIncorrectName(tName) {
if util.IsInCorrectIdentifierName(tName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(tName)
return
}
specs := stmt.Specs
for _, spec := range specs {
if spec.NewTable != nil {
ntName := spec.NewTable.Name.String()
if isIncorrectName(ntName) {
if util.IsInCorrectIdentifierName(ntName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(ntName)
return
}
Expand All @@ -1217,15 +1217,15 @@ func (p *preprocessor) checkAlterTableGrammar(stmt *ast.AlterTableStmt) {
}
case ast.AlterTableAddStatistics, ast.AlterTableDropStatistics:
statsName := spec.Statistics.StatsName
if isIncorrectName(statsName) {
if util.IsInCorrectIdentifierName(statsName) {
msg := fmt.Sprintf("Incorrect statistics name: %s", statsName)
p.err = ErrInternal.GenWithStack(msg)
return
}
case ast.AlterTableAddPartitions:
for _, def := range spec.PartDefinitions {
pName := def.Name.String()
if isIncorrectName(pName) {
if util.IsInCorrectIdentifierName(pName) {
p.err = dbterror.ErrWrongPartitionName.GenWithStackByArgs()
return
}
Expand Down Expand Up @@ -1334,7 +1334,7 @@ func checkReferInfoForTemporaryTable(tableMetaInfo *model.TableInfo) error {
func checkColumn(colDef *ast.ColumnDef) error {
// Check column name.
cName := colDef.Name.Name.String()
if isIncorrectName(cName) {
if util.IsInCorrectIdentifierName(cName) {
return dbterror.ErrWrongColumnName.GenWithStackByArgs(cName)
}

Expand Down Expand Up @@ -1457,18 +1457,6 @@ func isInvalidDefaultValue(colDef *ast.ColumnDef) bool {
return false
}

// isIncorrectName checks if the identifier is incorrect.
// See https://dev.mysql.com/doc/refman/5.7/en/identifiers.html
func isIncorrectName(name string) bool {
if len(name) == 0 {
return true
}
if name[len(name)-1] == ' ' {
return true
}
return false
}

// checkContainDotColumn checks field contains the table name.
// for example :create table t (c1.c2 int default null).
func (p *preprocessor) checkContainDotColumn(stmt *ast.CreateTableStmt) {
Expand Down Expand Up @@ -1683,7 +1671,7 @@ func (p *preprocessor) resolveAlterTableStmt(node *ast.AlterTableStmt) {

func (p *preprocessor) resolveCreateSequenceStmt(stmt *ast.CreateSequenceStmt) {
sName := stmt.Name.Name.String()
if isIncorrectName(sName) {
if util.IsInCorrectIdentifierName(sName) {
p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(sName)
return
}
Expand Down
1 change: 1 addition & 0 deletions server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ go_library(
"//parser/ast",
"//parser/auth",
"//parser/charset",
"//parser/model",
"//parser/mysql",
"//parser/terror",
"//planner/core",
Expand Down
18 changes: 16 additions & 2 deletions server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ import (
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
plannercore "github.com/pingcap/tidb/planner/core"
Expand Down Expand Up @@ -977,17 +978,30 @@ func (cc *clientConn) Run(ctx context.Context) {
close(cc.quit)
}()

parentCtx := ctx
var traceInfo *model.TraceInfo
// Usually, client connection status changes between [dispatching] <=> [reading].
// When some event happens, server may notify this client connection by setting
// the status to special values, for example: kill or graceful shutdown.
// The client connection would detect the events when it fails to change status
// by CAS operation, it would then take some actions accordingly.
for {
sessVars := cc.ctx.GetSessionVars()
if alias := sessVars.SessionAlias; traceInfo == nil || traceInfo.SessionAlias != alias {
// We should reset the context trace info when traceInfo not inited or session alias changed.
traceInfo = &model.TraceInfo{
ConnectionID: cc.connectionID,
SessionAlias: alias,
}
ctx = logutil.WithSessionAlias(parentCtx, sessVars.SessionAlias)
ctx = tracing.ContextWithTraceInfo(ctx, traceInfo)
}

// Close connection between txn when we are going to shutdown server.
// Note the current implementation when shutting down, for an idle connection, the connection may block at readPacket()
// consider provider a way to close the connection directly after sometime if we can not read any data.
if cc.server.inShutdownMode.Load() {
if !cc.ctx.GetSessionVars().InTxn() {
if !sessVars.InTxn() {
return
}
}
Expand Down Expand Up @@ -1216,7 +1230,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error {
defer task.End()

trace.Log(ctx, "sql", lc.String())
ctx = logutil.WithTraceLogger(ctx, cc.connectionID)
ctx = logutil.WithTraceLogger(ctx, tracing.TraceInfoFromContext(ctx))

taskID := *(*uint64)(unsafe.Pointer(task))
ctx = pprof.WithLabels(ctx, pprof.Labels("trace", strconv.FormatUint(taskID, 10)))
Expand Down
2 changes: 1 addition & 1 deletion session/test/variable/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ go_test(
"variable_test.go",
],
flaky = True,
shard_count = 21,
shard_count = 22,
deps = [
"//config",
"//kv",
Expand Down
21 changes: 21 additions & 0 deletions session/test/variable/variable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -608,3 +608,24 @@ func TestSysdateIsNow(t *testing.T) {
tk.MustQuery("show variables like '%tidb_sysdate_is_now%'").Check(testkit.Rows("tidb_sysdate_is_now ON"))
require.True(t, tk.Session().GetSessionVars().SysdateIsNow)
}

func TestSessionAlias(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows(""))
// normal set
tk.MustExec("set @@tidb_session_alias='alias123'")
tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows("alias123"))
// set a long value
val := "0123456789012345678901234567890123456789012345678901234567890123456789"
tk.MustExec("set @@tidb_session_alias=?", val)
tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows(val[:64]))
// an invalid value
err := tk.ExecToErr("set @@tidb_session_alias='abc '")
require.EqualError(t, err, "[variable:1231]Incorrect value for variable @@tidb_session_alias 'abc '")
tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows(val[:64]))
// reset to empty
tk.MustExec("set @@tidb_session_alias=''")
tk.MustQuery("select @@tidb_session_alias").Check(testkit.Rows(""))
}
3 changes: 3 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1534,6 +1534,9 @@ type SessionVars struct {
// When set to true, skip missing partition stats and continue to merge other partition stats to global stats.
// When set to false, give up merging partition stats to global stats.
SkipMissingPartitionStats bool

// SessionAlias is the identifier of the session
SessionAlias string
}

// GetOptimizerFixControlMap returns the specified value of the optimizer fix control.
Expand Down
20 changes: 20 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
_ "github.com/pingcap/tidb/types/parser_driver" // for parser driver
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/gctuner"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -2792,6 +2793,25 @@ var defaultSysVars = []*SysVar{
}, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) {
return BoolToOnOff(EnableCheckConstraint.Load()), nil
}},
{Scope: ScopeSession, Name: TiDBSessionAlias, Value: "", Type: TypeStr,
Validation: func(s *SessionVars, normalizedValue string, originalValue string, _ ScopeFlag) (string, error) {
if len(normalizedValue) > 64 {
s.StmtCtx.AppendWarning(ErrTruncatedWrongValue.GenWithStackByArgs(TiDBSessionAlias, originalValue))
normalizedValue = normalizedValue[:64]
}

if len(normalizedValue) > 0 && util.IsInCorrectIdentifierName(normalizedValue) {
return "", ErrWrongValueForVar.GenWithStack("Incorrect value for variable @@%s '%s'", TiDBSessionAlias, normalizedValue)
}

return normalizedValue, nil
},
SetSession: func(vars *SessionVars, s string) error {
vars.SessionAlias = s
return nil
}, GetSession: func(vars *SessionVars) (string, error) {
return vars.SessionAlias, nil
}},
}

func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error {
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1083,6 +1083,8 @@ const (
// When set to true, skip missing partition stats and continue to merge other partition stats to global stats.
// When set to false, give up merging partition stats to global stats.
TiDBSkipMissingPartitionStats = "tidb_skip_missing_partition_stats"
// TiDBSessionAlias indicates the alias of a session which is used for tracing.
TiDBSessionAlias = "tidb_session_alias"
)

// TiDB intentional limits
Expand Down
3 changes: 3 additions & 0 deletions util/logutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ go_library(
importpath = "github.com/pingcap/tidb/util/logutil",
visibility = ["//visibility:public"],
deps = [
"//parser/model",
"@com_github_golang_protobuf//proto",
"@com_github_grpc_ecosystem_go_grpc_middleware//logging/zap",
"@com_github_opentracing_opentracing_go//:opentracing-go",
Expand All @@ -35,7 +36,9 @@ go_test(
flaky = True,
deps = [
"//kv",
"//parser/model",
"//testkit/testsetup",
"@com_github_google_uuid//:uuid",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_log//:log",
"@com_github_stretchr_testify//require",
Expand Down
Loading

0 comments on commit ddb7b36

Please sign in to comment.