Skip to content

Commit

Permalink
*: move config file option query-log-max-len to sysvar (#34305)
Browse files Browse the repository at this point in the history
ref #33769
  • Loading branch information
espresso98 authored May 3, 2022
1 parent fbe6015 commit 698a4ff
Show file tree
Hide file tree
Showing 7 changed files with 56 additions and 18 deletions.
3 changes: 1 addition & 2 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -415,7 +415,6 @@ type Log struct {
SlowQueryFile string `toml:"slow-query-file" json:"slow-query-file"`
SlowThreshold uint64 `toml:"slow-threshold" json:"slow-threshold"`
ExpensiveThreshold uint `toml:"expensive-threshold" json:"expensive-threshold"`
QueryLogMaxLen uint64 `toml:"query-log-max-len" json:"query-log-max-len"`
RecordPlanInSlowLog uint32 `toml:"record-plan-in-slow-log" json:"record-plan-in-slow-log"`
}

Expand Down Expand Up @@ -771,7 +770,6 @@ var defaultConf = Config{
EnableErrorStack: nbUnset, // If both options are nbUnset, getDisableErrorStack() returns true
EnableTimestamp: nbUnset,
DisableTimestamp: nbUnset, // If both options are nbUnset, getDisableTimestamp() returns false
QueryLogMaxLen: logutil.DefaultQueryLogMaxLen,
RecordPlanInSlowLog: logutil.DefaultRecordPlanInSlowLog,
EnableSlowLog: *NewAtomicBool(logutil.DefaultTiDBEnableSlowLog),
},
Expand Down Expand Up @@ -931,6 +929,7 @@ var deprecatedConfig = map[string]struct{}{
"stmt-summary.refresh-interval": {},
"stmt-summary.history-size": {},
"mem-quota-query": {},
"query-log-max-len": {},
}

func isAllDeprecatedConfigItems(items []string) bool {
Expand Down
1 change: 0 additions & 1 deletion config/config_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ var (
"OOMAction": {},
"TiKVClient.StoreLimit": {},
"Log.Level": {},
"Log.QueryLogMaxLen": {},
"Log.ExpensiveThreshold": {},
"Instance.SlowThreshold": {},
"Instance.CheckMb4ValueInUTF8": {},
Expand Down
7 changes: 5 additions & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -929,9 +929,12 @@ func (a *ExecStmt) logAudit() {
// FormatSQL is used to format the original SQL, e.g. truncating long SQL, appending prepared arguments.
func FormatSQL(sql string) stringutil.StringerFunc {
return func() string {
cfg := config.GetGlobalConfig()
length := len(sql)
if maxQueryLen := atomic.LoadUint64(&cfg.Log.QueryLogMaxLen); uint64(length) > maxQueryLen {
maxQueryLen := variable.QueryLogMaxLen.Load()
if maxQueryLen <= 0 {
return QueryReplacer.Replace(sql) // no limit
}
if int32(length) > maxQueryLen {
sql = fmt.Sprintf("%.*q(len:%d)", maxQueryLen, sql, length)
}
return QueryReplacer.Replace(sql)
Expand Down
13 changes: 13 additions & 0 deletions executor/adapter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@ import (
"testing"
"time"

"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
)
Expand All @@ -40,3 +42,14 @@ func TestQueryTime(t *testing.T) {
costTime = time.Since(tk.Session().GetSessionVars().StartTime)
require.Less(t, costTime, time.Second)
}

func TestFormatSQL(t *testing.T) {
val := executor.FormatSQL("aaaa")
require.Equal(t, "aaaa", val.String())
variable.QueryLogMaxLen.Store(0)
val = executor.FormatSQL("aaaaaaaaaaaaaaaaaaaa")
require.Equal(t, "aaaaaaaaaaaaaaaaaaaa", val.String())
variable.QueryLogMaxLen.Store(5)
val = executor.FormatSQL("aaaaaaaaaaaaaaaaaaaa")
require.Equal(t, "\"aaaaa\"(len:20)", val.String())
}
12 changes: 6 additions & 6 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -283,12 +283,6 @@ var defaultSysVars = []*SysVar{
s.MetricSchemaRangeDuration = TidbOptInt64(val, DefTiDBMetricSchemaRangeDuration)
return nil
}},
{Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, skipInit: true, SetSession: func(s *SessionVars, val string) error {
atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(TidbOptInt64(val, logutil.DefaultQueryLogMaxLen)))
return nil
}, GetSession: func(s *SessionVars) (string, error) {
return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10), nil
}},
{Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error {
s.FoundInPlanCache = TiDBOptOn(val)
return nil
Expand Down Expand Up @@ -683,6 +677,12 @@ var defaultSysVars = []*SysVar{
return nil
},
},
{Scope: ScopeGlobal, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(DefTiDBQueryLogMaxLen), Type: TypeInt, MinValue: 0, MaxValue: 1073741824, SetGlobal: func(s *SessionVars, val string) error {
QueryLogMaxLen.Store(int32(TidbOptInt64(val, DefTiDBQueryLogMaxLen)))
return nil
}, GetGlobal: func(s *SessionVars) (string, error) {
return fmt.Sprint(QueryLogMaxLen.Load()), nil
}},

/* The system variables below have GLOBAL and SESSION scope */
{Scope: ScopeGlobal | ScopeSession, Name: SQLSelectLimit, Value: "18446744073709551615", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error {
Expand Down
30 changes: 26 additions & 4 deletions sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -606,10 +606,6 @@ func TestInstanceScopedVars(t *testing.T) {
require.NoError(t, err)
require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.EnableSlowLog.Load()), val)

val, err = GetSessionOrGlobalSystemVar(vars, TiDBQueryLogMaxLen)
require.NoError(t, err)
require.Equal(t, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10), val)

val, err = GetSessionOrGlobalSystemVar(vars, TiDBCheckMb4ValueInUTF8)
require.NoError(t, err)
require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Instance.CheckMb4ValueInUTF8.Load()), val)
Expand Down Expand Up @@ -946,3 +942,29 @@ func TestTiDBMemQuotaQuery(t *testing.T) {
require.NoError(t, err)
}
}

func TestTiDBQueryLogMaxLen(t *testing.T) {
sv := GetSysVar(TiDBQueryLogMaxLen)
vars := NewSessionVars()

newVal := 32 * 1024 * 1024
val, err := sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal)
require.Equal(t, val, "33554432")
require.NoError(t, err)

// out of range
newVal = 1073741825
expected := 1073741824
val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal)
// expected to truncate
require.Equal(t, val, fmt.Sprintf("%d", expected))
require.NoError(t, err)

// min value out of range
newVal = -2
expected = 0
val, err = sv.Validate(vars, fmt.Sprintf("%d", newVal), ScopeGlobal)
// expected to set to min value
require.Equal(t, val, fmt.Sprintf("%d", expected))
require.NoError(t, err)
}
8 changes: 5 additions & 3 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,9 +186,6 @@ const (
// TiDBEnableSlowLog enables TiDB to log slow queries.
TiDBEnableSlowLog = "tidb_enable_slow_log"

// TiDBQueryLogMaxLen is used to set the max length of the query in the log.
TiDBQueryLogMaxLen = "tidb_query_log_max_len"

// TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value.
TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8"

Expand Down Expand Up @@ -633,6 +630,9 @@ const (

// TiDBBatchPendingTiFlashCount indicates the maximum count of non-available TiFlash tables.
TiDBBatchPendingTiFlashCount = "tidb_batch_pending_tiflash_count"

// TiDBQueryLogMaxLen is used to set the max length of the query in the log.
TiDBQueryLogMaxLen = "tidb_query_log_max_len"
)

// TiDB vars that have only global scope
Expand Down Expand Up @@ -840,12 +840,14 @@ const (
DefTiDBGCMaxWaitTime = 24 * 60 * 60
DefMaxAllowedPacket uint64 = 67108864
DefTiDBMemQuotaQuery = 1073741824 // 1GB
DefTiDBQueryLogMaxLen = 4096
)

// Process global variables.
var (
ProcessGeneralLog = atomic.NewBool(false)
GlobalLogMaxDays = atomic.NewInt32(int32(config.GetGlobalConfig().Log.File.MaxDays))
QueryLogMaxLen = atomic.NewInt32(DefTiDBQueryLogMaxLen)
EnablePProfSQLCPU = atomic.NewBool(false)
ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount
ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize
Expand Down

0 comments on commit 698a4ff

Please sign in to comment.